diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 0cd9cb89d5f88..7c16ba30720b3 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -1,3 +1,20 @@ +# +# 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. +# + # Imports from base R # Do not include stats:: "rpois", "runif" - causes error at runtime importFrom("methods", "setGeneric", "setMethod", "setOldClass") @@ -47,7 +64,8 @@ exportMethods("glm", "spark.kstest", "spark.logit", "spark.randomForest", - "spark.gbt") + "spark.gbt", + "spark.bisectingKmeans") # Job group lifecycle management methods export("setJobGroup", @@ -94,6 +112,7 @@ exportMethods("arrange", "freqItems", "gapply", "gapplyCollect", + "getNumPartitions", "group_by", "groupBy", "head", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 3d912c9fa32ed..bfec3245cf52a 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1717,6 +1717,23 @@ getColumn <- function(x, c) { column(callJMethod(x@sdf, "col", c)) } +setColumn <- function(x, c, value) { + if (class(value) != "Column" && !is.null(value)) { + if (isAtomicLengthOne(value)) { + value <- lit(value) + } else { + stop("value must be a Column, literal value as atomic in length of 1, or NULL") + } + } + + if (is.null(value)) { + nx <- drop(x, c) + } else { + nx <- withColumn(x, c, value) + } + nx +} + #' @param name name of a Column (without being wrapped by \code{""}). #' @rdname select #' @name $ @@ -1735,19 +1752,7 @@ setMethod("$", signature(x = "SparkDataFrame"), #' @note $<- since 1.4.0 setMethod("$<-", signature(x = "SparkDataFrame"), function(x, name, value) { - if (class(value) != "Column" && !is.null(value)) { - if (isAtomicLengthOne(value)) { - value <- lit(value) - } else { - stop("value must be a Column, literal value as atomic in length of 1, or NULL") - } - } - - if (is.null(value)) { - nx <- drop(x, name) - } else { - nx <- withColumn(x, name, value) - } + nx <- setColumn(x, name, value) x@sdf <- nx@sdf x }) @@ -1767,6 +1772,21 @@ setMethod("[[", signature(x = "SparkDataFrame", i = "numericOrcharacter"), getColumn(x, i) }) +#' @rdname subset +#' @name [[<- +#' @aliases [[<-,SparkDataFrame,numericOrcharacter-method +#' @note [[<- since 2.1.1 +setMethod("[[<-", signature(x = "SparkDataFrame", i = "numericOrcharacter"), + function(x, i, value) { + if (is.numeric(i)) { + cols <- columns(x) + i <- cols[[i]] + } + nx <- setColumn(x, i, value) + x@sdf <- nx@sdf + x + }) + #' @rdname subset #' @name [ #' @aliases [,SparkDataFrame-method @@ -1811,14 +1831,19 @@ setMethod("[", signature(x = "SparkDataFrame"), #' Return subsets of SparkDataFrame according to given conditions #' @param x a SparkDataFrame. #' @param i,subset (Optional) a logical expression to filter on rows. +#' For extract operator [[ and replacement operator [[<-, the indexing parameter for +#' a single Column. #' @param j,select expression for the single Column or a list of columns to select from the SparkDataFrame. #' @param drop if TRUE, a Column will be returned if the resulting dataset has only one column. #' Otherwise, a SparkDataFrame will always be returned. +#' @param value a Column or an atomic vector in the length of 1 as literal value, or \code{NULL}. +#' If \code{NULL}, the specified Column is dropped. #' @param ... currently not used. #' @return A new SparkDataFrame containing only the rows that meet the condition with selected columns. #' @export #' @family SparkDataFrame functions #' @aliases subset,SparkDataFrame-method +#' @seealso \link{withColumn} #' @rdname subset #' @name subset #' @family subsetting functions @@ -1836,6 +1861,10 @@ setMethod("[", signature(x = "SparkDataFrame"), #' subset(df, df$age %in% c(19, 30), 1:2) #' subset(df, df$age %in% c(19), select = c(1,2)) #' subset(df, select = c(1,2)) +#' # Columns can be selected and set +#' df[["age"]] <- 23 +#' df[[1]] <- df$age +#' df[[2]] <- NULL # drop column #' } #' @note subset since 1.5.0 setMethod("subset", signature(x = "SparkDataFrame"), @@ -1960,7 +1989,7 @@ setMethod("selectExpr", #' @aliases withColumn,SparkDataFrame,character-method #' @rdname withColumn #' @name withColumn -#' @seealso \link{rename} \link{mutate} +#' @seealso \link{rename} \link{mutate} \link{subset} #' @export #' @examples #'\dontrun{ @@ -1971,6 +2000,10 @@ setMethod("selectExpr", #' # Replace an existing column #' newDF2 <- withColumn(newDF, "newCol", newDF$col1) #' newDF3 <- withColumn(newDF, "newCol", 42) +#' # Use extract operator to set an existing or new column +#' df[["age"]] <- 23 +#' df[[2]] <- df$col1 +#' df[[2]] <- NULL # drop column #' } #' @note withColumn since 1.4.0 setMethod("withColumn", @@ -3406,3 +3439,26 @@ setMethod("randomSplit", } sapply(sdfs, dataFrame) }) + +#' getNumPartitions +#' +#' Return the number of partitions +#' +#' @param x A SparkDataFrame +#' @family SparkDataFrame functions +#' @aliases getNumPartitions,SparkDataFrame-method +#' @rdname getNumPartitions +#' @name getNumPartitions +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- createDataFrame(cars, numPartitions = 2) +#' getNumPartitions(df) +#' } +#' @note getNumPartitions since 2.1.1 +setMethod("getNumPartitions", + signature(x = "SparkDataFrame"), + function(x) { + callJMethod(callJMethod(x@sdf, "rdd"), "getNumPartitions") + }) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 0f1162fec1df9..91bab332c2864 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -313,7 +313,7 @@ setMethod("checkpoint", #' @rdname getNumPartitions #' @aliases getNumPartitions,RDD-method #' @noRd -setMethod("getNumPartitions", +setMethod("getNumPartitionsRDD", signature(x = "RDD"), function(x) { callJMethod(getJRDD(x), "getNumPartitions") @@ -329,7 +329,7 @@ setMethod("numPartitions", signature(x = "RDD"), function(x) { .Deprecated("getNumPartitions") - getNumPartitions(x) + getNumPartitionsRDD(x) }) #' Collect elements of an RDD @@ -460,7 +460,7 @@ setMethod("countByValue", signature(x = "RDD"), function(x) { ones <- lapply(x, function(item) { list(item, 1L) }) - collectRDD(reduceByKey(ones, `+`, getNumPartitions(x))) + collectRDD(reduceByKey(ones, `+`, getNumPartitionsRDD(x))) }) #' Apply a function to all elements @@ -780,7 +780,7 @@ setMethod("takeRDD", resList <- list() index <- -1 jrdd <- getJRDD(x) - numPartitions <- getNumPartitions(x) + numPartitions <- getNumPartitionsRDD(x) serializedModeRDD <- getSerializedMode(x) # TODO(shivaram): Collect more than one partition based on size @@ -846,7 +846,7 @@ setMethod("firstRDD", #' @noRd setMethod("distinctRDD", signature(x = "RDD"), - function(x, numPartitions = SparkR:::getNumPartitions(x)) { + function(x, numPartitions = SparkR:::getNumPartitionsRDD(x)) { identical.mapped <- lapply(x, function(x) { list(x, NULL) }) reduced <- reduceByKey(identical.mapped, function(x, y) { x }, @@ -1053,7 +1053,7 @@ setMethod("coalesce", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions, shuffle = FALSE) { numPartitions <- numToInt(numPartitions) - if (shuffle || numPartitions > SparkR:::getNumPartitions(x)) { + if (shuffle || numPartitions > SparkR:::getNumPartitionsRDD(x)) { func <- function(partIndex, part) { set.seed(partIndex) # partIndex as seed start <- as.integer(base::sample(numPartitions, 1) - 1) @@ -1143,7 +1143,7 @@ setMethod("saveAsTextFile", #' @noRd setMethod("sortBy", signature(x = "RDD", func = "function"), - function(x, func, ascending = TRUE, numPartitions = SparkR:::getNumPartitions(x)) { + function(x, func, ascending = TRUE, numPartitions = SparkR:::getNumPartitionsRDD(x)) { values(sortByKey(keyBy(x, func), ascending, numPartitions)) }) @@ -1175,7 +1175,7 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { resList <- list() index <- -1 jrdd <- getJRDD(newRdd) - numPartitions <- getNumPartitions(newRdd) + numPartitions <- getNumPartitionsRDD(newRdd) serializedModeRDD <- getSerializedMode(newRdd) while (TRUE) { @@ -1407,7 +1407,7 @@ setMethod("setName", setMethod("zipWithUniqueId", signature(x = "RDD"), function(x) { - n <- getNumPartitions(x) + n <- getNumPartitionsRDD(x) partitionFunc <- function(partIndex, part) { mapply( @@ -1450,7 +1450,7 @@ setMethod("zipWithUniqueId", setMethod("zipWithIndex", signature(x = "RDD"), function(x) { - n <- getNumPartitions(x) + n <- getNumPartitionsRDD(x) if (n > 1) { nums <- collectRDD(lapplyPartition(x, function(part) { @@ -1566,8 +1566,8 @@ setMethod("unionRDD", setMethod("zipRDD", signature(x = "RDD", other = "RDD"), function(x, other) { - n1 <- getNumPartitions(x) - n2 <- getNumPartitions(other) + n1 <- getNumPartitionsRDD(x) + n2 <- getNumPartitionsRDD(other) if (n1 != n2) { stop("Can only zip RDDs which have the same number of partitions.") } @@ -1637,7 +1637,7 @@ setMethod("cartesian", #' @noRd setMethod("subtract", signature(x = "RDD", other = "RDD"), - function(x, other, numPartitions = SparkR:::getNumPartitions(x)) { + function(x, other, numPartitions = SparkR:::getNumPartitionsRDD(x)) { mapFunction <- function(e) { list(e, NA) } rdd1 <- map(x, mapFunction) rdd2 <- map(other, mapFunction) @@ -1671,7 +1671,7 @@ setMethod("subtract", #' @noRd setMethod("intersection", signature(x = "RDD", other = "RDD"), - function(x, other, numPartitions = SparkR:::getNumPartitions(x)) { + function(x, other, numPartitions = SparkR:::getNumPartitionsRDD(x)) { rdd1 <- map(x, function(v) { list(v, NA) }) rdd2 <- map(other, function(v) { list(v, NA) }) @@ -1714,7 +1714,7 @@ setMethod("zipPartitions", if (length(rrdds) == 1) { return(rrdds[[1]]) } - nPart <- sapply(rrdds, getNumPartitions) + nPart <- sapply(rrdds, getNumPartitionsRDD) if (length(unique(nPart)) != 1) { stop("Can only zipPartitions RDDs which have the same number of partitions.") } diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 499c7b279ea9d..0307bac349ec1 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -138,9 +138,9 @@ setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") }) # @export setGeneric("name", function(x) { standardGeneric("name") }) -# @rdname getNumPartitions +# @rdname getNumPartitionsRDD # @export -setGeneric("getNumPartitions", function(x) { standardGeneric("getNumPartitions") }) +setGeneric("getNumPartitionsRDD", function(x) { standardGeneric("getNumPartitionsRDD") }) # @rdname getNumPartitions # @export @@ -492,6 +492,10 @@ setGeneric("gapply", function(x, ...) { standardGeneric("gapply") }) #' @export setGeneric("gapplyCollect", function(x, ...) { standardGeneric("gapplyCollect") }) +# @rdname getNumPartitions +# @export +setGeneric("getNumPartitions", function(x) { standardGeneric("getNumPartitions") }) + #' @rdname summary #' @export setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) @@ -1338,6 +1342,11 @@ setGeneric("rbind", signature = "...") #' @export setGeneric("spark.als", function(data, ...) { standardGeneric("spark.als") }) +#' @rdname spark.bisectingKmeans +#' @export +setGeneric("spark.bisectingKmeans", + function(data, formula, ...) { standardGeneric("spark.bisectingKmeans") }) + #' @rdname spark.gaussianMixture #' @export setGeneric("spark.gaussianMixture", diff --git a/R/pkg/R/install.R b/R/pkg/R/install.R index cb6bbe5946b14..72386e68de4b3 100644 --- a/R/pkg/R/install.R +++ b/R/pkg/R/install.R @@ -54,7 +54,7 @@ #' } #' @param overwrite If \code{TRUE}, download and overwrite the existing tar file in localDir #' and force re-install Spark (in case the local directory or file is corrupted) -#' @return \code{install.spark} returns the local directory where Spark is found or installed +#' @return the (invisible) local directory where Spark is found or installed #' @rdname install.spark #' @name install.spark #' @aliases install.spark @@ -115,17 +115,35 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL, } else { if (releaseUrl != "") { message("Downloading from alternate URL:\n- ", releaseUrl) - downloadUrl(releaseUrl, packageLocalPath, paste0("Fetch failed from ", releaseUrl)) + success <- downloadUrl(releaseUrl, packageLocalPath) + if (!success) { + unlink(packageLocalPath) + stop(paste0("Fetch failed from ", releaseUrl)) + } } else { robustDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) } } message(sprintf("Installing to %s", localDir)) - untar(tarfile = packageLocalPath, exdir = localDir) - if (!tarExists || overwrite) { + # There are two ways untar can fail - untar could stop() on errors like incomplete block on file + # or, tar command can return failure code + success <- tryCatch(untar(tarfile = packageLocalPath, exdir = localDir) == 0, + error = function(e) { + message(e) + message() + FALSE + }, + warning = function(w) { + # Treat warning as error, add an empty line with message() + message(w) + message() + FALSE + }) + if (!tarExists || overwrite || !success) { unlink(packageLocalPath) } + if (!success) stop("Extract archive failed.") message("DONE.") Sys.setenv(SPARK_HOME = packageLocalDir) message(paste("SPARK_HOME set to", packageLocalDir)) @@ -135,8 +153,7 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL, robustDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) { # step 1: use user-provided url if (!is.null(mirrorUrl)) { - msg <- sprintf("Use user-provided mirror site: %s.", mirrorUrl) - message(msg) + message("Use user-provided mirror site: ", mirrorUrl) success <- directDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) if (success) { @@ -156,7 +173,7 @@ robustDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, pa packageName, packageLocalPath) if (success) return() } else { - message("Unable to find preferred mirror site.") + message("Unable to download from preferred mirror site: ", mirrorUrl) } # step 3: use backup option @@ -165,8 +182,11 @@ robustDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, pa success <- directDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) if (success) { - return(packageLocalPath) + return() } else { + # remove any partially downloaded file + unlink(packageLocalPath) + message("Unable to download from default mirror site: ", mirrorUrl) msg <- sprintf(paste("Unable to download Spark %s for Hadoop %s.", "Please check network connection, Hadoop version,", "or provide other mirror sites."), @@ -201,14 +221,20 @@ directDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, pa msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion), packageRemotePath) message(msg) - downloadUrl(packageRemotePath, packageLocalPath, paste0("Fetch failed from ", mirrorUrl)) + downloadUrl(packageRemotePath, packageLocalPath) } -downloadUrl <- function(remotePath, localPath, errorMessage) { +downloadUrl <- function(remotePath, localPath) { isFail <- tryCatch(download.file(remotePath, localPath), error = function(e) { - message(errorMessage) - print(e) + message(e) + message() + TRUE + }, + warning = function(w) { + # Treat warning as error, add an empty line with message() + message(w) + message() TRUE }) !isFail @@ -234,10 +260,9 @@ sparkCachePath <- function() { if (.Platform$OS.type == "windows") { winAppPath <- Sys.getenv("LOCALAPPDATA", unset = NA) if (is.na(winAppPath)) { - msg <- paste("%LOCALAPPDATA% not found.", + stop(paste("%LOCALAPPDATA% not found.", "Please define the environment variable", - "or restart and enter an installation path in localDir.") - stop(msg) + "or restart and enter an installation path in localDir.")) } else { path <- file.path(winAppPath, "Apache", "Spark", "Cache") } diff --git a/R/pkg/R/mllib_classification.R b/R/pkg/R/mllib_classification.R index 8da84499dfc09..fee4a4cc9ff27 100644 --- a/R/pkg/R/mllib_classification.R +++ b/R/pkg/R/mllib_classification.R @@ -41,7 +41,7 @@ setClass("NaiveBayesModel", representation(jobj = "jobj")) #' Logistic Regression Model #' -#' Fits an logistic regression model against a Spark DataFrame. It supports "binomial": Binary logistic regression +#' Fits an logistic regression model against a SparkDataFrame. It supports "binomial": Binary logistic regression #' with pivoting; "multinomial": Multinomial logistic (softmax) regression without pivoting, similar to glmnet. #' Users can print, make predictions on the produced model and save the model to the input path. #' diff --git a/R/pkg/R/mllib_clustering.R b/R/pkg/R/mllib_clustering.R index fb8d9e75ad091..e384c7398b22f 100644 --- a/R/pkg/R/mllib_clustering.R +++ b/R/pkg/R/mllib_clustering.R @@ -17,6 +17,13 @@ # mllib_clustering.R: Provides methods for MLlib clustering algorithms integration +#' S4 class that represents a BisectingKMeansModel +#' +#' @param jobj a Java object reference to the backing Scala BisectingKMeansModel +#' @export +#' @note BisectingKMeansModel since 2.2.0 +setClass("BisectingKMeansModel", representation(jobj = "jobj")) + #' S4 class that represents a GaussianMixtureModel #' #' @param jobj a Java object reference to the backing Scala GaussianMixtureModel @@ -38,9 +45,151 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' @note LDAModel since 2.1.0 setClass("LDAModel", representation(jobj = "jobj")) +#' Bisecting K-Means Clustering Model +#' +#' Fits a bisecting k-means clustering model against a SparkDataFrame. +#' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make +#' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' Note that the response variable of formula is empty in spark.bisectingKmeans. +#' @param k the desired number of leaf clusters. Must be > 1. +#' The actual number could be smaller if there are no divisible leaf clusters. +#' @param maxIter maximum iteration number. +#' @param seed the random seed. +#' @param minDivisibleClusterSize The minimum number of points (if greater than or equal to 1.0) +#' or the minimum proportion of points (if less than 1.0) of a divisible cluster. +#' Note that it is an expert parameter. The default value should be good enough +#' for most cases. +#' @param ... additional argument(s) passed to the method. +#' @return \code{spark.bisectingKmeans} returns a fitted bisecting k-means model. +#' @rdname spark.bisectingKmeans +#' @aliases spark.bisectingKmeans,SparkDataFrame,formula-method +#' @name spark.bisectingKmeans +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' df <- createDataFrame(iris) +#' model <- spark.bisectingKmeans(df, Sepal_Length ~ Sepal_Width, k = 4) +#' summary(model) +#' +#' # get fitted result from a bisecting k-means model +#' fitted.model <- fitted(model, "centers") +#' showDF(fitted.model) +#' +#' # fitted values on training data +#' fitted <- predict(model, df) +#' head(select(fitted, "Sepal_Length", "prediction")) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and print +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.bisectingKmeans since 2.2.0 +#' @seealso \link{predict}, \link{read.ml}, \link{write.ml} +setMethod("spark.bisectingKmeans", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, k = 4, maxIter = 20, seed = NULL, minDivisibleClusterSize = 1.0) { + formula <- paste0(deparse(formula), collapse = "") + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + jobj <- callJStatic("org.apache.spark.ml.r.BisectingKMeansWrapper", "fit", + data@sdf, formula, as.integer(k), as.integer(maxIter), + seed, as.numeric(minDivisibleClusterSize)) + new("BisectingKMeansModel", jobj = jobj) + }) + +# Get the summary of a bisecting k-means model + +#' @param object a fitted bisecting k-means model. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes the model's \code{k} (number of cluster centers), +#' \code{coefficients} (model cluster centers), +#' \code{size} (number of data points in each cluster), \code{cluster} +#' (cluster centers of the transformed data; cluster is NULL if is.loaded is TRUE), +#' and \code{is.loaded} (whether the model is loaded from a saved file). +#' @rdname spark.bisectingKmeans +#' @export +#' @note summary(BisectingKMeansModel) since 2.2.0 +setMethod("summary", signature(object = "BisectingKMeansModel"), + function(object) { + jobj <- object@jobj + is.loaded <- callJMethod(jobj, "isLoaded") + features <- callJMethod(jobj, "features") + coefficients <- callJMethod(jobj, "coefficients") + k <- callJMethod(jobj, "k") + size <- callJMethod(jobj, "size") + coefficients <- t(matrix(coefficients, ncol = k)) + colnames(coefficients) <- unlist(features) + rownames(coefficients) <- 1:k + cluster <- if (is.loaded) { + NULL + } else { + dataFrame(callJMethod(jobj, "cluster")) + } + list(k = k, coefficients = coefficients, size = size, + cluster = cluster, is.loaded = is.loaded) + }) + +# Predicted values based on a bisecting k-means model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns the predicted values based on a bisecting k-means model. +#' @rdname spark.bisectingKmeans +#' @export +#' @note predict(BisectingKMeansModel) since 2.2.0 +setMethod("predict", signature(object = "BisectingKMeansModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +#' Get fitted result from a bisecting k-means model +#' +#' Get fitted result from a bisecting k-means model. +#' Note: A saved-loaded model does not support this method. +#' +#' @param method type of fitted results, \code{"centers"} for cluster centers +#' or \code{"classes"} for assigned classes. +#' @return \code{fitted} returns a SparkDataFrame containing fitted values. +#' @rdname spark.bisectingKmeans +#' @export +#' @note fitted since 2.2.0 +setMethod("fitted", signature(object = "BisectingKMeansModel"), + function(object, method = c("centers", "classes")) { + method <- match.arg(method) + jobj <- object@jobj + is.loaded <- callJMethod(jobj, "isLoaded") + if (is.loaded) { + stop("Saved-loaded bisecting k-means model does not support 'fitted' method") + } else { + dataFrame(callJMethod(jobj, "fitted", method)) + } + }) + +# Save fitted MLlib model to the input path + +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.bisectingKmeans +#' @export +#' @note write.ml(BisectingKMeansModel, character) since 2.2.0 +setMethod("write.ml", signature(object = "BisectingKMeansModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + #' Multivariate Gaussian Mixture Model (GMM) #' -#' Fits multivariate gaussian mixture model against a Spark DataFrame, similarly to R's +#' Fits multivariate gaussian mixture model against a SparkDataFrame, similarly to R's #' mvnormalmixEM(). Users can call \code{summary} to print a summary of the fitted model, #' \code{predict} to make predictions on new data, and \code{write.ml}/\code{read.ml} #' to save/load fitted models. @@ -98,7 +247,7 @@ setMethod("spark.gaussianMixture", signature(data = "SparkDataFrame", formula = #' @param object a fitted gaussian mixture model. #' @return \code{summary} returns summary of the fitted model, which is a list. #' The list includes the model's \code{lambda} (lambda), \code{mu} (mu), -#' \code{sigma} (sigma), and \code{posterior} (posterior). +#' \code{sigma} (sigma), \code{loglik} (loglik), and \code{posterior} (posterior). #' @aliases spark.gaussianMixture,SparkDataFrame,formula-method #' @rdname spark.gaussianMixture #' @export @@ -112,6 +261,7 @@ setMethod("summary", signature(object = "GaussianMixtureModel"), sigmaList <- callJMethod(jobj, "sigma") k <- callJMethod(jobj, "k") dim <- callJMethod(jobj, "dim") + loglik <- callJMethod(jobj, "logLikelihood") mu <- c() for (i in 1 : k) { start <- (i - 1) * dim + 1 @@ -129,7 +279,7 @@ setMethod("summary", signature(object = "GaussianMixtureModel"), } else { dataFrame(callJMethod(jobj, "posterior")) } - list(lambda = lambda, mu = mu, sigma = sigma, + list(lambda = lambda, mu = mu, sigma = sigma, loglik = loglik, posterior = posterior, is.loaded = is.loaded) }) @@ -164,7 +314,7 @@ setMethod("write.ml", signature(object = "GaussianMixtureModel", path = "charact #' K-Means Clustering Model #' -#' Fits a k-means clustering model against a Spark DataFrame, similarly to R's kmeans(). +#' Fits a k-means clustering model against a SparkDataFrame, similarly to R's kmeans(). #' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make #' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. #' diff --git a/R/pkg/R/mllib_regression.R b/R/pkg/R/mllib_regression.R index 0e07d3bfd899c..79086003463e7 100644 --- a/R/pkg/R/mllib_regression.R +++ b/R/pkg/R/mllib_regression.R @@ -41,7 +41,7 @@ setClass("IsotonicRegressionModel", representation(jobj = "jobj")) #' Generalized Linear Models #' -#' Fits generalized linear model against a Spark DataFrame. +#' Fits generalized linear model against a SparkDataFrame. #' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make #' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. #' @@ -259,7 +259,7 @@ setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", pat #' Isotonic Regression Model #' -#' Fits an Isotonic Regression model against a Spark DataFrame, similarly to R's isoreg(). +#' Fits an Isotonic Regression model against a SparkDataFrame, similarly to R's isoreg(). #' Users can print, make predictions on the produced model and save the model to the input path. #' #' @param data SparkDataFrame for training. diff --git a/R/pkg/R/mllib_utils.R b/R/pkg/R/mllib_utils.R index 720ee41c58065..29c44739233d7 100644 --- a/R/pkg/R/mllib_utils.R +++ b/R/pkg/R/mllib_utils.R @@ -32,8 +32,8 @@ #' @rdname write.ml #' @name write.ml #' @export -#' @seealso \link{spark.glm}, \link{glm}, -#' @seealso \link{spark.als}, \link{spark.gaussianMixture}, \link{spark.gbt}, \link{spark.isoreg}, +#' @seealso \link{spark.als}, \link{spark.bisectingKmeans}, \link{spark.gaussianMixture}, +#' @seealso \link{spark.gbt}, \link{spark.glm}, \link{glm}, \link{spark.isoreg}, #' @seealso \link{spark.kmeans}, #' @seealso \link{spark.lda}, \link{spark.logit}, \link{spark.mlp}, \link{spark.naiveBayes}, #' @seealso \link{spark.randomForest}, \link{spark.survreg}, @@ -47,8 +47,8 @@ NULL #' @rdname predict #' @name predict #' @export -#' @seealso \link{spark.glm}, \link{glm}, -#' @seealso \link{spark.als}, \link{spark.gaussianMixture}, \link{spark.gbt}, \link{spark.isoreg}, +#' @seealso \link{spark.als}, \link{spark.bisectingKmeans}, \link{spark.gaussianMixture}, +#' @seealso \link{spark.gbt}, \link{spark.glm}, \link{glm}, \link{spark.isoreg}, #' @seealso \link{spark.kmeans}, #' @seealso \link{spark.logit}, \link{spark.mlp}, \link{spark.naiveBayes}, #' @seealso \link{spark.randomForest}, \link{spark.survreg} @@ -113,6 +113,8 @@ read.ml <- function(path) { new("GBTRegressionModel", jobj = jobj) } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.GBTClassifierWrapper")) { new("GBTClassificationModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.BisectingKMeansWrapper")) { + new("BisectingKMeansModel", jobj = jobj) } else { stop("Unsupported model: ", jobj) } diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 4dee3245f9b75..8fa21be3076b5 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -780,7 +780,7 @@ setMethod("cogroup", #' @noRd setMethod("sortByKey", signature(x = "RDD"), - function(x, ascending = TRUE, numPartitions = SparkR:::getNumPartitions(x)) { + function(x, ascending = TRUE, numPartitions = SparkR:::getNumPartitionsRDD(x)) { rangeBounds <- list() if (numPartitions > 1) { @@ -850,7 +850,7 @@ setMethod("sortByKey", #' @noRd setMethod("subtractByKey", signature(x = "RDD", other = "RDD"), - function(x, other, numPartitions = SparkR:::getNumPartitions(x)) { + function(x, other, numPartitions = SparkR:::getNumPartitionsRDD(x)) { filterFunction <- function(elem) { iters <- elem[[2]] (length(iters[[1]]) > 0) && (length(iters[[2]]) == 0) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 74b3e502eb7ca..1f7848f2b413f 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -756,12 +756,17 @@ varargsToJProperties <- function(...) { props } -launchScript <- function(script, combinedArgs, capture = FALSE) { +launchScript <- function(script, combinedArgs, wait = FALSE) { if (.Platform$OS.type == "windows") { scriptWithArgs <- paste(script, combinedArgs, sep = " ") - shell(scriptWithArgs, translate = TRUE, wait = capture, intern = capture) # nolint + # on Windows, intern = F seems to mean output to the console. (documentation on this is missing) + shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) # nolint } else { - system2(script, combinedArgs, wait = capture, stdout = capture) + # http://stat.ethz.ch/R-manual/R-devel/library/base/html/system2.html + # stdout = F means discard output + # stdout = "" means to its console (default) + # Note that the console of this child process might not be the same as the running R process. + system2(script, combinedArgs, stdout = "", wait = wait) } } diff --git a/R/pkg/inst/tests/testthat/test_Windows.R b/R/pkg/inst/tests/testthat/test_Windows.R index 8813e18a1fa4d..e8d983426a676 100644 --- a/R/pkg/inst/tests/testthat/test_Windows.R +++ b/R/pkg/inst/tests/testthat/test_Windows.R @@ -20,7 +20,7 @@ test_that("sparkJars tag in SparkContext", { if (.Platform$OS.type != "windows") { skip("This test is only for Windows, skipped") } - testOutput <- launchScript("ECHO", "a/b/c", capture = TRUE) + testOutput <- launchScript("ECHO", "a/b/c", wait = TRUE) abcPath <- testOutput[1] expect_equal(abcPath, "a\\b\\c") }) diff --git a/R/pkg/inst/tests/testthat/test_mllib_clustering.R b/R/pkg/inst/tests/testthat/test_mllib_clustering.R index cfbdea5c041fb..aad834bb64b8e 100644 --- a/R/pkg/inst/tests/testthat/test_mllib_clustering.R +++ b/R/pkg/inst/tests/testthat/test_mllib_clustering.R @@ -27,6 +27,46 @@ absoluteSparkPath <- function(x) { file.path(sparkHome, x) } +test_that("spark.bisectingKmeans", { + newIris <- iris + newIris$Species <- NULL + training <- suppressWarnings(createDataFrame(newIris)) + + take(training, 1) + + model <- spark.bisectingKmeans(data = training, ~ .) + sample <- take(select(predict(model, training), "prediction"), 1) + expect_equal(typeof(sample$prediction), "integer") + expect_equal(sample$prediction, 1) + + # Test fitted works on Bisecting KMeans + fitted.model <- fitted(model) + expect_equal(sort(collect(distinct(select(fitted.model, "prediction")))$prediction), + c(0, 1, 2, 3)) + + # Test summary works on KMeans + summary.model <- summary(model) + cluster <- summary.model$cluster + k <- summary.model$k + expect_equal(k, 4) + expect_equal(sort(collect(distinct(select(cluster, "prediction")))$prediction), + c(0, 1, 2, 3)) + + # Test model save/load + modelPath <- tempfile(pattern = "spark-bisectingkmeans", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + summary2 <- summary(model2) + expect_equal(sort(unlist(summary.model$size)), sort(unlist(summary2$size))) + expect_equal(summary.model$coefficients, summary2$coefficients) + expect_true(!summary.model$is.loaded) + expect_true(summary2$is.loaded) + + unlink(modelPath) +}) + test_that("spark.gaussianMixture", { # R code to reproduce the result. # nolint start @@ -56,6 +96,10 @@ test_that("spark.gaussianMixture", { # [,1] [,2] # [1,] 0.2961543 0.160783 # [2,] 0.1607830 1.008878 + # + #' model$loglik + # + # [1] -46.89499 # nolint end data <- list(list(-0.6264538, 0.1836433), list(-0.8356286, 1.5952808), list(0.3295078, -0.8204684), list(0.4874291, 0.7383247), @@ -72,9 +116,11 @@ test_that("spark.gaussianMixture", { rMu <- c(0.11731091, -0.06192351, 10.363673, 9.897081) rSigma <- c(0.62049934, 0.06880802, 0.06880802, 1.27431874, 0.2961543, 0.160783, 0.1607830, 1.008878) + rLoglik <- -46.89499 expect_equal(stats$lambda, rLambda, tolerance = 1e-3) expect_equal(unlist(stats$mu), rMu, tolerance = 1e-3) expect_equal(unlist(stats$sigma), rSigma, tolerance = 1e-3) + expect_equal(unlist(stats$loglik), rLoglik, tolerance = 1e-3) p <- collect(select(predict(model, df), "prediction")) expect_equal(p$prediction, c(0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1)) @@ -88,6 +134,7 @@ test_that("spark.gaussianMixture", { expect_equal(stats$lambda, stats2$lambda) expect_equal(unlist(stats$mu), unlist(stats2$mu)) expect_equal(unlist(stats$sigma), unlist(stats2$sigma)) + expect_equal(unlist(stats$loglik), unlist(stats2$loglik)) unlink(modelPath) }) diff --git a/R/pkg/inst/tests/testthat/test_rdd.R b/R/pkg/inst/tests/testthat/test_rdd.R index 2c41a6b075b47..ceb31bd896133 100644 --- a/R/pkg/inst/tests/testthat/test_rdd.R +++ b/R/pkg/inst/tests/testthat/test_rdd.R @@ -29,8 +29,8 @@ intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) intRdd <- parallelize(sc, intPairs, 2L) test_that("get number of partitions in RDD", { - expect_equal(getNumPartitions(rdd), 2) - expect_equal(getNumPartitions(intRdd), 2) + expect_equal(getNumPartitionsRDD(rdd), 2) + expect_equal(getNumPartitionsRDD(intRdd), 2) }) test_that("first on RDD", { @@ -305,18 +305,18 @@ test_that("repartition/coalesce on RDDs", { # repartition r1 <- repartitionRDD(rdd, 2) - expect_equal(getNumPartitions(r1), 2L) + expect_equal(getNumPartitionsRDD(r1), 2L) count <- length(collectPartition(r1, 0L)) expect_true(count >= 8 && count <= 12) r2 <- repartitionRDD(rdd, 6) - expect_equal(getNumPartitions(r2), 6L) + expect_equal(getNumPartitionsRDD(r2), 6L) count <- length(collectPartition(r2, 0L)) expect_true(count >= 0 && count <= 4) # coalesce r3 <- coalesce(rdd, 1) - expect_equal(getNumPartitions(r3), 1L) + expect_equal(getNumPartitionsRDD(r3), 1L) count <- length(collectPartition(r3, 0L)) expect_equal(count, 20) }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 26017427ab5e7..417a03ff61827 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -196,18 +196,18 @@ test_that("create DataFrame from RDD", { expect_equal(dtypes(df), list(c("name", "string"), c("age", "int"), c("height", "float"))) expect_equal(as.list(collect(where(df, df$name == "John"))), list(name = "John", age = 19L, height = 176.5)) - expect_equal(getNumPartitions(toRDD(df)), 1) + expect_equal(getNumPartitions(df), 1) df <- as.DataFrame(cars, numPartitions = 2) - expect_equal(getNumPartitions(toRDD(df)), 2) + expect_equal(getNumPartitions(df), 2) df <- createDataFrame(cars, numPartitions = 3) - expect_equal(getNumPartitions(toRDD(df)), 3) + expect_equal(getNumPartitions(df), 3) # validate limit by num of rows df <- createDataFrame(cars, numPartitions = 60) - expect_equal(getNumPartitions(toRDD(df)), 50) + expect_equal(getNumPartitions(df), 50) # validate when 1 < (length(coll) / numSlices) << length(coll) df <- createDataFrame(cars, numPartitions = 20) - expect_equal(getNumPartitions(toRDD(df)), 20) + expect_equal(getNumPartitions(df), 20) df <- as.DataFrame(data.frame(0)) expect_is(df, "SparkDataFrame") @@ -215,7 +215,7 @@ test_that("create DataFrame from RDD", { expect_is(df, "SparkDataFrame") df <- as.DataFrame(data.frame(0), numPartitions = 2) # no data to partition, goes to 1 - expect_equal(getNumPartitions(toRDD(df)), 1) + expect_equal(getNumPartitions(df), 1) setHiveContext(sc) sql("CREATE TABLE people (name string, age double, height float)") @@ -234,7 +234,7 @@ test_that("createDataFrame uses files for large objects", { conf <- callJMethod(sparkSession, "conf") callJMethod(conf, "set", "spark.r.maxAllocationLimit", "100") df <- suppressWarnings(createDataFrame(iris, numPartitions = 3)) - expect_equal(getNumPartitions(toRDD(df)), 3) + expect_equal(getNumPartitions(df), 3) # Resetting the conf back to default value callJMethod(conf, "set", "spark.r.maxAllocationLimit", toString(.Machine$integer.max / 10)) @@ -1021,6 +1021,9 @@ test_that("select operators", { df$age2 <- df$age * 2 expect_equal(columns(df), c("name", "age", "age2")) expect_equal(count(where(df, df$age2 == df$age * 2)), 2) + df$age2 <- df[["age"]] * 3 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age * 3)), 2) df$age2 <- 21 expect_equal(columns(df), c("name", "age", "age2")) @@ -1033,6 +1036,23 @@ test_that("select operators", { expect_error(df$age3 <- c(22, NA), "value must be a Column, literal value as atomic in length of 1, or NULL") + df[["age2"]] <- 23 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == 23)), 3) + + df[[3]] <- 24 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == 24)), 3) + + df[[3]] <- df$age + expect_equal(count(where(df, df$age2 == df$age)), 2) + + df[["age2"]] <- df[["name"]] + expect_equal(count(where(df, df$age2 == df$name)), 3) + + expect_error(df[["age3"]] <- c(22, 23), + "value must be a Column, literal value as atomic in length of 1, or NULL") + # Test parameter drop expect_equal(class(df[, 1]) == "SparkDataFrame", T) expect_equal(class(df[, 1, drop = T]) == "Column", T) diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index 6f11c5c51676f..36a78477dc261 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -10,6 +10,23 @@ vignette: > \usepackage[utf8]{inputenc} --- + + ## Overview SparkR is an R package that provides a light-weight frontend to use Apache Spark from R. With Spark `r packageVersion("SparkR")`, SparkR provides a distributed data frame implementation that supports data processing operations like selection, filtering, aggregation etc. and distributed machine learning using [MLlib](http://spark.apache.org/mllib/). @@ -906,9 +923,9 @@ The main method calls of actual computation happen in the Spark JVM of the drive Two kinds of RPCs are supported in the SparkR JVM backend: method invocation and creating new objects. Method invocation can be done in two ways. -* `sparkR.invokeJMethod` takes a reference to an existing Java object and a list of arguments to be passed on to the method. +* `sparkR.callJMethod` takes a reference to an existing Java object and a list of arguments to be passed on to the method. -* `sparkR.invokeJStatic` takes a class name for static method and a list of arguments to be passed on to the method. +* `sparkR.callJStatic` takes a class name for static method and a list of arguments to be passed on to the method. The arguments are serialized using our custom wire format which is then deserialized on the JVM side. We then use Java reflection to invoke the appropriate method. diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java new file mode 100644 index 0000000000000..980525dbf04e0 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java @@ -0,0 +1,128 @@ +/* + * 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.network.crypto; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.security.GeneralSecurityException; +import java.security.Key; +import javax.crypto.KeyGenerator; +import javax.crypto.Mac; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.sasl.SaslClientBootstrap; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Bootstraps a {@link TransportClient} by performing authentication using Spark's auth protocol. + * + * This bootstrap falls back to using the SASL bootstrap if the server throws an error during + * authentication, and the configuration allows it. This is used for backwards compatibility + * with external shuffle services that do not support the new protocol. + * + * It also automatically falls back to SASL if the new encryption backend is disabled, so that + * callers only need to install this bootstrap when authentication is enabled. + */ +public class AuthClientBootstrap implements TransportClientBootstrap { + + private static final Logger LOG = LoggerFactory.getLogger(AuthClientBootstrap.class); + + private final TransportConf conf; + private final String appId; + private final String authUser; + private final SecretKeyHolder secretKeyHolder; + + public AuthClientBootstrap( + TransportConf conf, + String appId, + SecretKeyHolder secretKeyHolder) { + this.conf = conf; + // TODO: right now this behaves like the SASL backend, because when executors start up + // they don't necessarily know the app ID. So they send a hardcoded "user" that is defined + // in the SecurityManager, which will also always return the same secret (regardless of the + // user name). All that's needed here is for this "user" to match on both sides, since that's + // required by the protocol. At some point, though, it would be better for the actual app ID + // to be provided here. + this.appId = appId; + this.authUser = secretKeyHolder.getSaslUser(appId); + this.secretKeyHolder = secretKeyHolder; + } + + @Override + public void doBootstrap(TransportClient client, Channel channel) { + if (!conf.encryptionEnabled()) { + LOG.debug("AES encryption disabled, using old auth protocol."); + doSaslAuth(client, channel); + return; + } + + try { + doSparkAuth(client, channel); + } catch (GeneralSecurityException | IOException e) { + throw Throwables.propagate(e); + } catch (RuntimeException e) { + // There isn't a good exception that can be caught here to know whether it's really + // OK to switch back to SASL (because the server doesn't speak the new protocol). So + // try it anyway, and in the worst case things will fail again. + if (conf.saslFallback()) { + LOG.warn("New auth protocol failed, trying SASL.", e); + doSaslAuth(client, channel); + } else { + throw e; + } + } + } + + private void doSparkAuth(TransportClient client, Channel channel) + throws GeneralSecurityException, IOException { + + AuthEngine engine = new AuthEngine(authUser, secretKeyHolder.getSecretKey(authUser), conf); + try { + ClientChallenge challenge = engine.challenge(); + ByteBuf challengeData = Unpooled.buffer(challenge.encodedLength()); + challenge.encode(challengeData); + + ByteBuffer responseData = client.sendRpcSync(challengeData.nioBuffer(), + conf.authRTTimeoutMs()); + ServerResponse response = ServerResponse.decodeMessage(responseData); + + engine.validate(response); + engine.sessionCipher().addToChannel(channel); + } finally { + engine.close(); + } + } + + private void doSaslAuth(TransportClient client, Channel channel) { + SaslClientBootstrap sasl = new SaslClientBootstrap(conf, appId, secretKeyHolder); + sasl.doBootstrap(client, channel); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java new file mode 100644 index 0000000000000..b769ebeba36cc --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java @@ -0,0 +1,284 @@ +/* + * 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.network.crypto; + +import java.io.Closeable; +import java.io.IOException; +import java.math.BigInteger; +import java.security.GeneralSecurityException; +import java.util.Arrays; +import java.util.Properties; +import javax.crypto.Cipher; +import javax.crypto.SecretKey; +import javax.crypto.SecretKeyFactory; +import javax.crypto.ShortBufferException; +import javax.crypto.spec.IvParameterSpec; +import javax.crypto.spec.PBEKeySpec; +import javax.crypto.spec.SecretKeySpec; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Bytes; +import org.apache.commons.crypto.cipher.CryptoCipher; +import org.apache.commons.crypto.cipher.CryptoCipherFactory; +import org.apache.commons.crypto.random.CryptoRandom; +import org.apache.commons.crypto.random.CryptoRandomFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.util.TransportConf; + +/** + * A helper class for abstracting authentication and key negotiation details. This is used by + * both client and server sides, since the operations are basically the same. + */ +class AuthEngine implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(AuthEngine.class); + private static final BigInteger ONE = new BigInteger(new byte[] { 0x1 }); + + private final byte[] appId; + private final char[] secret; + private final TransportConf conf; + private final Properties cryptoConf; + private final CryptoRandom random; + + private byte[] authNonce; + + @VisibleForTesting + byte[] challenge; + + private TransportCipher sessionCipher; + private CryptoCipher encryptor; + private CryptoCipher decryptor; + + AuthEngine(String appId, String secret, TransportConf conf) throws GeneralSecurityException { + this.appId = appId.getBytes(UTF_8); + this.conf = conf; + this.cryptoConf = conf.cryptoConf(); + this.secret = secret.toCharArray(); + this.random = CryptoRandomFactory.getCryptoRandom(cryptoConf); + } + + /** + * Create the client challenge. + * + * @return A challenge to be sent the remote side. + */ + ClientChallenge challenge() throws GeneralSecurityException, IOException { + this.authNonce = randomBytes(conf.encryptionKeyLength() / Byte.SIZE); + SecretKeySpec authKey = generateKey(conf.keyFactoryAlgorithm(), conf.keyFactoryIterations(), + authNonce, conf.encryptionKeyLength()); + initializeForAuth(conf.cipherTransformation(), authNonce, authKey); + + this.challenge = randomBytes(conf.encryptionKeyLength() / Byte.SIZE); + return new ClientChallenge(new String(appId, UTF_8), + conf.keyFactoryAlgorithm(), + conf.keyFactoryIterations(), + conf.cipherTransformation(), + conf.encryptionKeyLength(), + authNonce, + challenge(appId, authNonce, challenge)); + } + + /** + * Validates the client challenge, and create the encryption backend for the channel from the + * parameters sent by the client. + * + * @param clientChallenge The challenge from the client. + * @return A response to be sent to the client. + */ + ServerResponse respond(ClientChallenge clientChallenge) + throws GeneralSecurityException, IOException { + + SecretKeySpec authKey = generateKey(clientChallenge.kdf, clientChallenge.iterations, + clientChallenge.nonce, clientChallenge.keyLength); + initializeForAuth(clientChallenge.cipher, clientChallenge.nonce, authKey); + + byte[] challenge = validateChallenge(clientChallenge.nonce, clientChallenge.challenge); + byte[] response = challenge(appId, clientChallenge.nonce, rawResponse(challenge)); + byte[] sessionNonce = randomBytes(conf.encryptionKeyLength() / Byte.SIZE); + byte[] inputIv = randomBytes(conf.ivLength()); + byte[] outputIv = randomBytes(conf.ivLength()); + + SecretKeySpec sessionKey = generateKey(clientChallenge.kdf, clientChallenge.iterations, + sessionNonce, clientChallenge.keyLength); + this.sessionCipher = new TransportCipher(cryptoConf, clientChallenge.cipher, sessionKey, + inputIv, outputIv); + + // Note the IVs are swapped in the response. + return new ServerResponse(response, encrypt(sessionNonce), encrypt(outputIv), encrypt(inputIv)); + } + + /** + * Validates the server response and initializes the cipher to use for the session. + * + * @param serverResponse The response from the server. + */ + void validate(ServerResponse serverResponse) throws GeneralSecurityException { + byte[] response = validateChallenge(authNonce, serverResponse.response); + + byte[] expected = rawResponse(challenge); + Preconditions.checkArgument(Arrays.equals(expected, response)); + + byte[] nonce = decrypt(serverResponse.nonce); + byte[] inputIv = decrypt(serverResponse.inputIv); + byte[] outputIv = decrypt(serverResponse.outputIv); + + SecretKeySpec sessionKey = generateKey(conf.keyFactoryAlgorithm(), conf.keyFactoryIterations(), + nonce, conf.encryptionKeyLength()); + this.sessionCipher = new TransportCipher(cryptoConf, conf.cipherTransformation(), sessionKey, + inputIv, outputIv); + } + + TransportCipher sessionCipher() { + Preconditions.checkState(sessionCipher != null); + return sessionCipher; + } + + @Override + public void close() throws IOException { + // Close ciphers (by calling "doFinal()" with dummy data) and the random instance so that + // internal state is cleaned up. Error handling here is just for paranoia, and not meant to + // accurately report the errors when they happen. + RuntimeException error = null; + byte[] dummy = new byte[8]; + try { + doCipherOp(encryptor, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); + } + try { + doCipherOp(decryptor, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); + } + random.close(); + + if (error != null) { + throw error; + } + } + + @VisibleForTesting + byte[] challenge(byte[] appId, byte[] nonce, byte[] challenge) throws GeneralSecurityException { + return encrypt(Bytes.concat(appId, nonce, challenge)); + } + + @VisibleForTesting + byte[] rawResponse(byte[] challenge) { + BigInteger orig = new BigInteger(challenge); + BigInteger response = orig.add(ONE); + return response.toByteArray(); + } + + private byte[] decrypt(byte[] in) throws GeneralSecurityException { + return doCipherOp(decryptor, in, false); + } + + private byte[] encrypt(byte[] in) throws GeneralSecurityException { + return doCipherOp(encryptor, in, false); + } + + private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key) + throws GeneralSecurityException { + + // commons-crypto currently only supports ciphers that require an initial vector; so + // create a dummy vector so that we can initialize the ciphers. In the future, if + // different ciphers are supported, this will have to be configurable somehow. + byte[] iv = new byte[conf.ivLength()]; + System.arraycopy(nonce, 0, iv, 0, Math.min(nonce.length, iv.length)); + + encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv)); + + decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv)); + } + + /** + * Validates an encrypted challenge as defined in the protocol, and returns the byte array + * that corresponds to the actual challenge data. + */ + private byte[] validateChallenge(byte[] nonce, byte[] encryptedChallenge) + throws GeneralSecurityException { + + byte[] challenge = decrypt(encryptedChallenge); + checkSubArray(appId, challenge, 0); + checkSubArray(nonce, challenge, appId.length); + return Arrays.copyOfRange(challenge, appId.length + nonce.length, challenge.length); + } + + private SecretKeySpec generateKey(String kdf, int iterations, byte[] salt, int keyLength) + throws GeneralSecurityException { + + SecretKeyFactory factory = SecretKeyFactory.getInstance(kdf); + PBEKeySpec spec = new PBEKeySpec(secret, salt, iterations, keyLength); + + long start = System.nanoTime(); + SecretKey key = factory.generateSecret(spec); + long end = System.nanoTime(); + + LOG.debug("Generated key with {} iterations in {} us.", conf.keyFactoryIterations(), + (end - start) / 1000); + + return new SecretKeySpec(key.getEncoded(), conf.keyAlgorithm()); + } + + private byte[] doCipherOp(CryptoCipher cipher, byte[] in, boolean isFinal) + throws GeneralSecurityException { + + Preconditions.checkState(cipher != null); + + int scale = 1; + while (true) { + int size = in.length * scale; + byte[] buffer = new byte[size]; + try { + int outSize = isFinal ? cipher.doFinal(in, 0, in.length, buffer, 0) + : cipher.update(in, 0, in.length, buffer, 0); + if (outSize != buffer.length) { + byte[] output = new byte[outSize]; + System.arraycopy(buffer, 0, output, 0, output.length); + return output; + } else { + return buffer; + } + } catch (ShortBufferException e) { + // Try again with a bigger buffer. + scale *= 2; + } + } + } + + private byte[] randomBytes(int count) { + byte[] bytes = new byte[count]; + random.nextBytes(bytes); + return bytes; + } + + /** Checks that the "test" array is in the data array starting at the given offset. */ + private void checkSubArray(byte[] test, byte[] data, int offset) { + Preconditions.checkArgument(data.length >= test.length + offset); + for (int i = 0; i < test.length; i++) { + Preconditions.checkArgument(test[i] == data[i + offset]); + } + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java new file mode 100644 index 0000000000000..991d8ba95f5ee --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java @@ -0,0 +1,170 @@ +/* + * 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.network.crypto; + +import java.io.IOException; +import java.nio.ByteBuffer; +import javax.security.sasl.Sasl; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.sasl.SaslRpcHandler; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * RPC Handler which performs authentication using Spark's auth protocol before delegating to a + * child RPC handler. If the configuration allows, this handler will delegate messages to a SASL + * RPC handler for further authentication, to support for clients that do not support Spark's + * protocol. + * + * The delegate will only receive messages if the given connection has been successfully + * authenticated. A connection may be authenticated at most once. + */ +class AuthRpcHandler extends RpcHandler { + private static final Logger LOG = LoggerFactory.getLogger(AuthRpcHandler.class); + + /** Transport configuration. */ + private final TransportConf conf; + + /** The client channel. */ + private final Channel channel; + + /** + * RpcHandler we will delegate to for authenticated connections. When falling back to SASL + * this will be replaced with the SASL RPC handler. + */ + @VisibleForTesting + RpcHandler delegate; + + /** Class which provides secret keys which are shared by server and client on a per-app basis. */ + private final SecretKeyHolder secretKeyHolder; + + /** Whether auth is done and future calls should be delegated. */ + @VisibleForTesting + boolean doDelegate; + + AuthRpcHandler( + TransportConf conf, + Channel channel, + RpcHandler delegate, + SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.channel = channel; + this.delegate = delegate; + this.secretKeyHolder = secretKeyHolder; + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + if (doDelegate) { + delegate.receive(client, message, callback); + return; + } + + int position = message.position(); + int limit = message.limit(); + + ClientChallenge challenge; + try { + challenge = ClientChallenge.decodeMessage(message); + LOG.debug("Received new auth challenge for client {}.", channel.remoteAddress()); + } catch (RuntimeException e) { + if (conf.saslFallback()) { + LOG.warn("Failed to parse new auth challenge, reverting to SASL for client {}.", + channel.remoteAddress()); + delegate = new SaslRpcHandler(conf, channel, delegate, secretKeyHolder); + message.position(position); + message.limit(limit); + delegate.receive(client, message, callback); + doDelegate = true; + } else { + LOG.debug("Unexpected challenge message from client {}, closing channel.", + channel.remoteAddress()); + callback.onFailure(new IllegalArgumentException("Unknown challenge message.")); + channel.close(); + } + return; + } + + // Here we have the client challenge, so perform the new auth protocol and set up the channel. + AuthEngine engine = null; + try { + engine = new AuthEngine(challenge.appId, secretKeyHolder.getSecretKey(challenge.appId), conf); + ServerResponse response = engine.respond(challenge); + ByteBuf responseData = Unpooled.buffer(response.encodedLength()); + response.encode(responseData); + callback.onSuccess(responseData.nioBuffer()); + engine.sessionCipher().addToChannel(channel); + } catch (Exception e) { + // This is a fatal error: authentication has failed. Close the channel explicitly. + LOG.debug("Authentication failed for client {}, closing channel.", channel.remoteAddress()); + callback.onFailure(new IllegalArgumentException("Authentication failed.")); + channel.close(); + return; + } finally { + if (engine != null) { + try { + engine.close(); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + + LOG.debug("Authorization successful for client {}.", channel.remoteAddress()); + doDelegate = true; + } + + @Override + public void receive(TransportClient client, ByteBuffer message) { + delegate.receive(client, message); + } + + @Override + public StreamManager getStreamManager() { + return delegate.getStreamManager(); + } + + @Override + public void channelActive(TransportClient client) { + delegate.channelActive(client); + } + + @Override + public void channelInactive(TransportClient client) { + delegate.channelInactive(client); + } + + @Override + public void exceptionCaught(Throwable cause, TransportClient client) { + delegate.exceptionCaught(cause, client); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthServerBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthServerBootstrap.java new file mode 100644 index 0000000000000..77a2a6af4d134 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthServerBootstrap.java @@ -0,0 +1,55 @@ +/* + * 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.network.crypto; + +import io.netty.channel.Channel; + +import org.apache.spark.network.sasl.SaslServerBootstrap; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServerBootstrap; +import org.apache.spark.network.util.TransportConf; + +/** + * A bootstrap which is executed on a TransportServer's client channel once a client connects + * to the server, enabling authentication using Spark's auth protocol (and optionally SASL for + * clients that don't support the new protocol). + * + * It also automatically falls back to SASL if the new encryption backend is disabled, so that + * callers only need to install this bootstrap when authentication is enabled. + */ +public class AuthServerBootstrap implements TransportServerBootstrap { + + private final TransportConf conf; + private final SecretKeyHolder secretKeyHolder; + + public AuthServerBootstrap(TransportConf conf, SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.secretKeyHolder = secretKeyHolder; + } + + public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { + if (!conf.encryptionEnabled()) { + TransportServerBootstrap sasl = new SaslServerBootstrap(conf, secretKeyHolder); + return sasl.doBootstrap(channel, rpcHandler); + } + + return new AuthRpcHandler(conf, channel, rpcHandler, secretKeyHolder); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/ClientChallenge.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/ClientChallenge.java new file mode 100644 index 0000000000000..3312a5bd81a66 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/ClientChallenge.java @@ -0,0 +1,101 @@ +/* + * 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.network.crypto; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.protocol.Encoders; + +/** + * The client challenge message, used to initiate authentication. + * + * @see README.md + */ +public class ClientChallenge implements Encodable { + /** Serialization tag used to catch incorrect payloads. */ + private static final byte TAG_BYTE = (byte) 0xFA; + + public final String appId; + public final String kdf; + public final int iterations; + public final String cipher; + public final int keyLength; + public final byte[] nonce; + public final byte[] challenge; + + public ClientChallenge( + String appId, + String kdf, + int iterations, + String cipher, + int keyLength, + byte[] nonce, + byte[] challenge) { + this.appId = appId; + this.kdf = kdf; + this.iterations = iterations; + this.cipher = cipher; + this.keyLength = keyLength; + this.nonce = nonce; + this.challenge = challenge; + } + + @Override + public int encodedLength() { + return 1 + 4 + 4 + + Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(kdf) + + Encoders.Strings.encodedLength(cipher) + + Encoders.ByteArrays.encodedLength(nonce) + + Encoders.ByteArrays.encodedLength(challenge); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeByte(TAG_BYTE); + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, kdf); + buf.writeInt(iterations); + Encoders.Strings.encode(buf, cipher); + buf.writeInt(keyLength); + Encoders.ByteArrays.encode(buf, nonce); + Encoders.ByteArrays.encode(buf, challenge); + } + + public static ClientChallenge decodeMessage(ByteBuffer buffer) { + ByteBuf buf = Unpooled.wrappedBuffer(buffer); + + if (buf.readByte() != TAG_BYTE) { + throw new IllegalArgumentException("Expected ClientChallenge, received something else."); + } + + return new ClientChallenge( + Encoders.Strings.decode(buf), + Encoders.Strings.decode(buf), + buf.readInt(), + Encoders.Strings.decode(buf), + buf.readInt(), + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf)); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md b/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md new file mode 100644 index 0000000000000..14df703270498 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md @@ -0,0 +1,158 @@ +Spark Auth Protocol and AES Encryption Support +============================================== + +This file describes an auth protocol used by Spark as a more secure alternative to DIGEST-MD5. This +protocol is built on symmetric key encryption, based on the assumption that the two endpoints being +authenticated share a common secret, which is how Spark authentication currently works. The protocol +provides mutual authentication, meaning that after the negotiation both parties know that the remote +side knows the shared secret. The protocol is influenced by the ISO/IEC 9798 protocol, although it's +not an implementation of it. + +This protocol could be replaced with TLS PSK, except no PSK ciphers are available in the currently +released JREs. + +The protocol aims at solving the following shortcomings in Spark's current usage of DIGEST-MD5: + +- MD5 is an aging hash algorithm with known weaknesses, and a more secure alternative is desired. +- DIGEST-MD5 has a pre-defined set of ciphers for which it can generate keys. The only + viable, supported cipher these days is 3DES, and a more modern alternative is desired. +- Encrypting AES session keys with 3DES doesn't solve the issue, since the weakest link + in the negotiation would still be MD5 and 3DES. + +The protocol assumes that the shared secret is generated and distributed in a secure manner. + +The protocol always negotiates encryption keys. If encryption is not desired, the existing +SASL-based authentication, or no authentication at all, can be chosen instead. + +When messages are described below, it's expected that the implementation should support +arbitrary sizes for fields that don't have a fixed size. + +Client Challenge +---------------- + +The auth negotiation is started by the client. The client starts by generating an encryption +key based on the application's shared secret, and a nonce. + + KEY = KDF(SECRET, SALT, KEY_LENGTH) + +Where: +- KDF(): a key derivation function that takes a secret, a salt, a configurable number of + iterations, and a configurable key length. +- SALT: a byte sequence used to salt the key derivation function. +- KEY_LENGTH: length of the encryption key to generate. + + +The client generates a message with the following content: + + CLIENT_CHALLENGE = ( + APP_ID, + KDF, + ITERATIONS, + CIPHER, + KEY_LENGTH, + ANONCE, + ENC(APP_ID || ANONCE || CHALLENGE)) + +Where: + +- APP_ID: the application ID which the server uses to identify the shared secret. +- KDF: the key derivation function described above. +- ITERATIONS: number of iterations to run the KDF when generating keys. +- CIPHER: the cipher used to encrypt data. +- KEY_LENGTH: length of the encryption keys to generate, in bits. +- ANONCE: the nonce used as the salt when generating the auth key. +- ENC(): an encryption function that uses the cipher and the generated key. This function + will also be used in the definition of other messages below. +- CHALLENGE: a byte sequence used as a challenge to the server. +- ||: concatenation operator. + +When strings are used where byte arrays are expected, the UTF-8 representation of the string +is assumed. + +To respond to the challenge, the server should consider the byte array as representing an +arbitrary-length integer, and respond with the value of the integer plus one. + + +Server Response And Challenge +----------------------------- + +Once the client challenge is received, the server will generate the same auth key by +using the same algorithm the client has used. It will then verify the client challenge: +if the APP_ID and ANONCE fields match, the server knows that the client has the shared +secret. The server then creates a response to the client challenge, to prove that it also +has the secret key, and provides parameters to be used when creating the session key. + +The following describes the response from the server: + + SERVER_CHALLENGE = ( + ENC(APP_ID || ANONCE || RESPONSE), + ENC(SNONCE), + ENC(INIV), + ENC(OUTIV)) + +Where: + +- RESPONSE: the server's response to the client challenge. +- SNONCE: a nonce to be used as salt when generating the session key. +- INIV: initialization vector used to initialize the input channel of the client. +- OUTIV: initialization vector used to initialize the output channel of the client. + +At this point the server considers the client to be authenticated, and will try to +decrypt any data further sent by the client using the session key. + + +Default Algorithms +------------------ + +Configuration options are available for the KDF and cipher algorithms to use. + +The default KDF is "PBKDF2WithHmacSHA1". Users should be able to select any algorithm +from those supported by the `javax.crypto.SecretKeyFactory` class, as long as they support +PBEKeySpec when generating keys. The default number of iterations was chosen to take a +reasonable amount of time on modern CPUs. See the documentation in TransportConf for more +details. + +The default cipher algorithm is "AES/CTR/NoPadding". Users should be able to select any +algorithm supported by the commons-crypto library. It should allow the cipher to operate +in stream mode. + +The default key length is 128 (bits). + + +Implementation Details +---------------------- + +The commons-crypto library currently only supports AES ciphers, and requires an initialization +vector (IV). This first version of the protocol does not explicitly include the IV in the client +challenge message. Instead, the IV should be derived from the nonce, including the needed bytes, and +padding the IV with zeroes in case the nonce is not long enough. + +Future versions of the protocol might add support for new ciphers and explicitly include needed +configuration parameters in the messages. + + +Threat Assessment +----------------- + +The protocol is secure against different forms of attack: + +* Eavesdropping: the protocol is built on the assumption that it's computationally infeasible + to calculate the original secret from the encrypted messages. Neither the secret nor any + encryption keys are transmitted on the wire, encrypted or not. + +* Man-in-the-middle: because the protocol performs mutual authentication, both ends need to + know the shared secret to be able to decrypt session data. Even if an attacker is able to insert a + malicious "proxy" between endpoints, the attacker won't be able to read any of the data exchanged + between client and server, nor insert arbitrary commands for the server to execute. + +* Replay attacks: the use of nonces when generating keys prevents an attacker from being able to + just replay messages sniffed from the communication channel. + +An attacker may replay the client challenge and successfully "prove" to a server that it "knows" the +shared secret. But the attacker won't be able to decrypt the server's response, and thus won't be +able to generate a session key, which will make it hard to craft a valid, encrypted message that the +server will be able to understand. This will cause the server to close the connection as soon as the +attacker tries to send any command to the server. The attacker can just hold the channel open for +some time, which will be closed when the server times out the channel. These issues could be +separately mitigated by adding a shorter timeout for the first message after authentication, and +potentially by adding host blacklists if a possible attack is detected from a particular host. diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/ServerResponse.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/ServerResponse.java new file mode 100644 index 0000000000000..affdbf450b1d0 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/ServerResponse.java @@ -0,0 +1,85 @@ +/* + * 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.network.crypto; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.protocol.Encoders; + +/** + * Server's response to client's challenge. + * + * @see README.md + */ +public class ServerResponse implements Encodable { + /** Serialization tag used to catch incorrect payloads. */ + private static final byte TAG_BYTE = (byte) 0xFB; + + public final byte[] response; + public final byte[] nonce; + public final byte[] inputIv; + public final byte[] outputIv; + + public ServerResponse( + byte[] response, + byte[] nonce, + byte[] inputIv, + byte[] outputIv) { + this.response = response; + this.nonce = nonce; + this.inputIv = inputIv; + this.outputIv = outputIv; + } + + @Override + public int encodedLength() { + return 1 + + Encoders.ByteArrays.encodedLength(response) + + Encoders.ByteArrays.encodedLength(nonce) + + Encoders.ByteArrays.encodedLength(inputIv) + + Encoders.ByteArrays.encodedLength(outputIv); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeByte(TAG_BYTE); + Encoders.ByteArrays.encode(buf, response); + Encoders.ByteArrays.encode(buf, nonce); + Encoders.ByteArrays.encode(buf, inputIv); + Encoders.ByteArrays.encode(buf, outputIv); + } + + public static ServerResponse decodeMessage(ByteBuffer buffer) { + ByteBuf buf = Unpooled.wrappedBuffer(buffer); + + if (buf.readByte() != TAG_BYTE) { + throw new IllegalArgumentException("Expected ServerResponse, received something else."); + } + + return new ServerResponse( + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf)); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java similarity index 64% rename from common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java rename to common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java index 340986a63bbf3..7376d1ddc4818 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesCipher.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.network.sasl.aes; +package org.apache.spark.network.crypto; import java.io.IOException; import java.nio.ByteBuffer; @@ -25,115 +25,91 @@ import javax.crypto.spec.SecretKeySpec; import javax.crypto.spec.IvParameterSpec; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.*; import io.netty.util.AbstractReferenceCounted; -import org.apache.commons.crypto.cipher.CryptoCipherFactory; -import org.apache.commons.crypto.random.CryptoRandom; -import org.apache.commons.crypto.random.CryptoRandomFactory; import org.apache.commons.crypto.stream.CryptoInputStream; import org.apache.commons.crypto.stream.CryptoOutputStream; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.spark.network.util.ByteArrayReadableChannel; import org.apache.spark.network.util.ByteArrayWritableChannel; -import org.apache.spark.network.util.TransportConf; /** - * AES cipher for encryption and decryption. + * Cipher for encryption and decryption. */ -public class AesCipher { - private static final Logger logger = LoggerFactory.getLogger(AesCipher.class); - public static final String ENCRYPTION_HANDLER_NAME = "AesEncryption"; - public static final String DECRYPTION_HANDLER_NAME = "AesDecryption"; - public static final int STREAM_BUFFER_SIZE = 1024 * 32; - public static final String TRANSFORM = "AES/CTR/NoPadding"; - - private final SecretKeySpec inKeySpec; - private final IvParameterSpec inIvSpec; - private final SecretKeySpec outKeySpec; - private final IvParameterSpec outIvSpec; - private final Properties properties; - - public AesCipher(AesConfigMessage configMessage, TransportConf conf) throws IOException { - this.properties = conf.cryptoConf(); - this.inKeySpec = new SecretKeySpec(configMessage.inKey, "AES"); - this.inIvSpec = new IvParameterSpec(configMessage.inIv); - this.outKeySpec = new SecretKeySpec(configMessage.outKey, "AES"); - this.outIvSpec = new IvParameterSpec(configMessage.outIv); +public class TransportCipher { + @VisibleForTesting + static final String ENCRYPTION_HANDLER_NAME = "TransportEncryption"; + private static final String DECRYPTION_HANDLER_NAME = "TransportDecryption"; + private static final int STREAM_BUFFER_SIZE = 1024 * 32; + + private final Properties conf; + private final String cipher; + private final SecretKeySpec key; + private final byte[] inIv; + private final byte[] outIv; + + public TransportCipher( + Properties conf, + String cipher, + SecretKeySpec key, + byte[] inIv, + byte[] outIv) { + this.conf = conf; + this.cipher = cipher; + this.key = key; + this.inIv = inIv; + this.outIv = outIv; + } + + public String getCipherTransformation() { + return cipher; + } + + @VisibleForTesting + SecretKeySpec getKey() { + return key; + } + + /** The IV for the input channel (i.e. output channel of the remote side). */ + public byte[] getInputIv() { + return inIv; + } + + /** The IV for the output channel (i.e. input channel of the remote side). */ + public byte[] getOutputIv() { + return outIv; } - /** - * Create AES crypto output stream - * @param ch The underlying channel to write out. - * @return Return output crypto stream for encryption. - * @throws IOException - */ private CryptoOutputStream createOutputStream(WritableByteChannel ch) throws IOException { - return new CryptoOutputStream(TRANSFORM, properties, ch, outKeySpec, outIvSpec); + return new CryptoOutputStream(cipher, conf, ch, key, new IvParameterSpec(outIv)); } - /** - * Create AES crypto input stream - * @param ch The underlying channel used to read data. - * @return Return input crypto stream for decryption. - * @throws IOException - */ private CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { - return new CryptoInputStream(TRANSFORM, properties, ch, inKeySpec, inIvSpec); + return new CryptoInputStream(cipher, conf, ch, key, new IvParameterSpec(inIv)); } /** - * Add handlers to channel + * Add handlers to channel. + * * @param ch the channel for adding handlers * @throws IOException */ public void addToChannel(Channel ch) throws IOException { ch.pipeline() - .addFirst(ENCRYPTION_HANDLER_NAME, new AesEncryptHandler(this)) - .addFirst(DECRYPTION_HANDLER_NAME, new AesDecryptHandler(this)); - } - - /** - * Create the configuration message - * @param conf is the local transport configuration. - * @return Config message for sending. - */ - public static AesConfigMessage createConfigMessage(TransportConf conf) { - int keySize = conf.aesCipherKeySize(); - Properties properties = conf.cryptoConf(); - - try { - int paramLen = CryptoCipherFactory.getCryptoCipher(AesCipher.TRANSFORM, properties) - .getBlockSize(); - byte[] inKey = new byte[keySize]; - byte[] outKey = new byte[keySize]; - byte[] inIv = new byte[paramLen]; - byte[] outIv = new byte[paramLen]; - - CryptoRandom random = CryptoRandomFactory.getCryptoRandom(properties); - random.nextBytes(inKey); - random.nextBytes(outKey); - random.nextBytes(inIv); - random.nextBytes(outIv); - - return new AesConfigMessage(inKey, inIv, outKey, outIv); - } catch (Exception e) { - logger.error("AES config error", e); - throw Throwables.propagate(e); - } + .addFirst(ENCRYPTION_HANDLER_NAME, new EncryptionHandler(this)) + .addFirst(DECRYPTION_HANDLER_NAME, new DecryptionHandler(this)); } - private static class AesEncryptHandler extends ChannelOutboundHandlerAdapter { + private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { private final ByteArrayWritableChannel byteChannel; private final CryptoOutputStream cos; - AesEncryptHandler(AesCipher cipher) throws IOException { - byteChannel = new ByteArrayWritableChannel(AesCipher.STREAM_BUFFER_SIZE); + EncryptionHandler(TransportCipher cipher) throws IOException { + byteChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); cos = cipher.createOutputStream(byteChannel); } @@ -153,11 +129,11 @@ public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exce } } - private static class AesDecryptHandler extends ChannelInboundHandlerAdapter { + private static class DecryptionHandler extends ChannelInboundHandlerAdapter { private final CryptoInputStream cis; private final ByteArrayReadableChannel byteChannel; - AesDecryptHandler(AesCipher cipher) throws IOException { + DecryptionHandler(TransportCipher cipher) throws IOException { byteChannel = new ByteArrayReadableChannel(); cis = cipher.createInputStream(byteChannel); } @@ -207,7 +183,7 @@ private static class EncryptedMessage extends AbstractReferenceCounted implement this.buf = isByteBuf ? (ByteBuf) msg : null; this.region = isByteBuf ? null : (FileRegion) msg; this.transferred = 0; - this.byteRawChannel = new ByteArrayWritableChannel(AesCipher.STREAM_BUFFER_SIZE); + this.byteRawChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); this.cos = cos; this.byteEncChannel = ch; } diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java index a1bb453657460..647813772294e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java @@ -30,8 +30,6 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientBootstrap; -import org.apache.spark.network.sasl.aes.AesCipher; -import org.apache.spark.network.sasl.aes.AesConfigMessage; import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.TransportConf; @@ -42,24 +40,14 @@ public class SaslClientBootstrap implements TransportClientBootstrap { private static final Logger logger = LoggerFactory.getLogger(SaslClientBootstrap.class); - private final boolean encrypt; private final TransportConf conf; private final String appId; private final SecretKeyHolder secretKeyHolder; public SaslClientBootstrap(TransportConf conf, String appId, SecretKeyHolder secretKeyHolder) { - this(conf, appId, secretKeyHolder, false); - } - - public SaslClientBootstrap( - TransportConf conf, - String appId, - SecretKeyHolder secretKeyHolder, - boolean encrypt) { this.conf = conf; this.appId = appId; this.secretKeyHolder = secretKeyHolder; - this.encrypt = encrypt; } /** @@ -69,7 +57,7 @@ public SaslClientBootstrap( */ @Override public void doBootstrap(TransportClient client, Channel channel) { - SparkSaslClient saslClient = new SparkSaslClient(appId, secretKeyHolder, encrypt); + SparkSaslClient saslClient = new SparkSaslClient(appId, secretKeyHolder, conf.saslEncryption()); try { byte[] payload = saslClient.firstToken(); @@ -79,35 +67,19 @@ public void doBootstrap(TransportClient client, Channel channel) { msg.encode(buf); buf.writeBytes(msg.body().nioByteBuffer()); - ByteBuffer response = client.sendRpcSync(buf.nioBuffer(), conf.saslRTTimeoutMs()); + ByteBuffer response = client.sendRpcSync(buf.nioBuffer(), conf.authRTTimeoutMs()); payload = saslClient.response(JavaUtils.bufferToArray(response)); } client.setClientId(appId); - if (encrypt) { + if (conf.saslEncryption()) { if (!SparkSaslServer.QOP_AUTH_CONF.equals(saslClient.getNegotiatedProperty(Sasl.QOP))) { throw new RuntimeException( new SaslException("Encryption requests by negotiated non-encrypted connection.")); } - if (conf.aesEncryptionEnabled()) { - // Generate a request config message to send to server. - AesConfigMessage configMessage = AesCipher.createConfigMessage(conf); - ByteBuffer buf = configMessage.encodeMessage(); - - // Encrypted the config message. - byte[] toEncrypt = JavaUtils.bufferToArray(buf); - ByteBuffer encrypted = ByteBuffer.wrap(saslClient.wrap(toEncrypt, 0, toEncrypt.length)); - - client.sendRpcSync(encrypted, conf.saslRTTimeoutMs()); - AesCipher cipher = new AesCipher(configMessage, conf); - logger.info("Enabling AES cipher for client channel {}", client); - cipher.addToChannel(channel); - saslClient.dispose(); - } else { - SaslEncryption.addToChannel(channel, saslClient, conf.maxSaslEncryptedBlockSize()); - } + SaslEncryption.addToChannel(channel, saslClient, conf.maxSaslEncryptedBlockSize()); saslClient = null; logger.debug("Channel {} configured for encryption.", client); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java index b2f3ef214b7ac..0231428318add 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java @@ -29,8 +29,6 @@ import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.sasl.aes.AesCipher; -import org.apache.spark.network.sasl.aes.AesConfigMessage; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.util.JavaUtils; @@ -44,7 +42,7 @@ * Note that the authentication process consists of multiple challenge-response pairs, each of * which are individual RPCs. */ -class SaslRpcHandler extends RpcHandler { +public class SaslRpcHandler extends RpcHandler { private static final Logger logger = LoggerFactory.getLogger(SaslRpcHandler.class); /** Transport configuration. */ @@ -63,7 +61,7 @@ class SaslRpcHandler extends RpcHandler { private boolean isComplete; private boolean isAuthenticated; - SaslRpcHandler( + public SaslRpcHandler( TransportConf conf, Channel channel, RpcHandler delegate, @@ -122,37 +120,10 @@ public void receive(TransportClient client, ByteBuffer message, RpcResponseCallb return; } - if (!conf.aesEncryptionEnabled()) { - logger.debug("Enabling encryption for channel {}", client); - SaslEncryption.addToChannel(channel, saslServer, conf.maxSaslEncryptedBlockSize()); - complete(false); - return; - } - - // Extra negotiation should happen after authentication, so return directly while - // processing authenticate. - if (!isAuthenticated) { - logger.debug("SASL authentication successful for channel {}", client); - isAuthenticated = true; - return; - } - - // Create AES cipher when it is authenticated - try { - byte[] encrypted = JavaUtils.bufferToArray(message); - ByteBuffer decrypted = ByteBuffer.wrap(saslServer.unwrap(encrypted, 0 , encrypted.length)); - - AesConfigMessage configMessage = AesConfigMessage.decodeMessage(decrypted); - AesCipher cipher = new AesCipher(configMessage, conf); - - // Send response back to client to confirm that server accept config. - callback.onSuccess(JavaUtils.stringToBytes(AesCipher.TRANSFORM)); - logger.info("Enabling AES cipher for Server channel {}", client); - cipher.addToChannel(channel); - complete(true); - } catch (IOException ioe) { - throw new RuntimeException(ioe); - } + logger.debug("Enabling encryption for channel {}", client); + SaslEncryption.addToChannel(channel, saslServer, conf.maxSaslEncryptedBlockSize()); + complete(false); + return; } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java deleted file mode 100644 index 3ef6f74a1f89f..0000000000000 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/aes/AesConfigMessage.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.sasl.aes; - -import java.nio.ByteBuffer; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; - -import org.apache.spark.network.protocol.Encodable; -import org.apache.spark.network.protocol.Encoders; - -/** - * The AES cipher options for encryption negotiation. - */ -public class AesConfigMessage implements Encodable { - /** Serialization tag used to catch incorrect payloads. */ - private static final byte TAG_BYTE = (byte) 0xEB; - - public byte[] inKey; - public byte[] outKey; - public byte[] inIv; - public byte[] outIv; - - public AesConfigMessage(byte[] inKey, byte[] inIv, byte[] outKey, byte[] outIv) { - if (inKey == null || inIv == null || outKey == null || outIv == null) { - throw new IllegalArgumentException("Cipher Key or IV must not be null!"); - } - - this.inKey = inKey; - this.inIv = inIv; - this.outKey = outKey; - this.outIv = outIv; - } - - @Override - public int encodedLength() { - return 1 + - Encoders.ByteArrays.encodedLength(inKey) + Encoders.ByteArrays.encodedLength(outKey) + - Encoders.ByteArrays.encodedLength(inIv) + Encoders.ByteArrays.encodedLength(outIv); - } - - @Override - public void encode(ByteBuf buf) { - buf.writeByte(TAG_BYTE); - Encoders.ByteArrays.encode(buf, inKey); - Encoders.ByteArrays.encode(buf, inIv); - Encoders.ByteArrays.encode(buf, outKey); - Encoders.ByteArrays.encode(buf, outIv); - } - - /** - * Encode the config message. - * @return ByteBuffer which contains encoded config message. - */ - public ByteBuffer encodeMessage(){ - ByteBuffer buf = ByteBuffer.allocate(encodedLength()); - - ByteBuf wrappedBuf = Unpooled.wrappedBuffer(buf); - wrappedBuf.clear(); - encode(wrappedBuf); - - return buf; - } - - /** - * Decode the config message from buffer - * @param buffer the buffer contain encoded config message - * @return config message - */ - public static AesConfigMessage decodeMessage(ByteBuffer buffer) { - ByteBuf buf = Unpooled.wrappedBuffer(buffer); - - if (buf.readByte() != TAG_BYTE) { - throw new IllegalStateException("Expected AesConfigMessage, received something else" - + " (maybe your client does not have AES enabled?)"); - } - - byte[] outKey = Encoders.ByteArrays.decode(buf); - byte[] outIv = Encoders.ByteArrays.decode(buf); - byte[] inKey = Encoders.ByteArrays.decode(buf); - byte[] inIv = Encoders.ByteArrays.decode(buf); - return new AesConfigMessage(inKey, inIv, outKey, outIv); - } - -} diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 6a557fa75d064..c226d8f3bc8fa 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -117,9 +117,10 @@ public int numConnectionsPerPeer() { /** Send buffer size (SO_SNDBUF). */ public int sendBuf() { return conf.getInt(SPARK_NETWORK_IO_SENDBUFFER_KEY, -1); } - /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ - public int saslRTTimeoutMs() { - return (int) JavaUtils.timeStringAsSec(conf.get(SPARK_NETWORK_SASL_TIMEOUT_KEY, "30s")) * 1000; + /** Timeout for a single round trip of auth message exchange, in milliseconds. */ + public int authRTTimeoutMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.network.auth.rpcTimeout", + conf.get(SPARK_NETWORK_SASL_TIMEOUT_KEY, "30s"))) * 1000; } /** @@ -162,40 +163,95 @@ public int portMaxRetries() { } /** - * Maximum number of bytes to be encrypted at a time when SASL encryption is enabled. + * Enables strong encryption. Also enables the new auth protocol, used to negotiate keys. */ - public int maxSaslEncryptedBlockSize() { - return Ints.checkedCast(JavaUtils.byteStringAsBytes( - conf.get("spark.network.sasl.maxEncryptedBlockSize", "64k"))); + public boolean encryptionEnabled() { + return conf.getBoolean("spark.network.crypto.enabled", false); } /** - * Whether the server should enforce encryption on SASL-authenticated connections. + * The cipher transformation to use for encrypting session data. */ - public boolean saslServerAlwaysEncrypt() { - return conf.getBoolean("spark.network.sasl.serverAlwaysEncrypt", false); + public String cipherTransformation() { + return conf.get("spark.network.crypto.cipher", "AES/CTR/NoPadding"); + } + + /** + * The key generation algorithm. This should be an algorithm that accepts a "PBEKeySpec" + * as input. The default value (PBKDF2WithHmacSHA1) is available in Java 7. + */ + public String keyFactoryAlgorithm() { + return conf.get("spark.network.crypto.keyFactoryAlgorithm", "PBKDF2WithHmacSHA1"); + } + + /** + * How many iterations to run when generating keys. + * + * See some discussion about this at: http://security.stackexchange.com/q/3959 + * The default value was picked for speed, since it assumes that the secret has good entropy + * (128 bits by default), which is not generally the case with user passwords. + */ + public int keyFactoryIterations() { + return conf.getInt("spark.networy.crypto.keyFactoryIterations", 1024); + } + + /** + * Encryption key length, in bits. + */ + public int encryptionKeyLength() { + return conf.getInt("spark.network.crypto.keyLength", 128); + } + + /** + * Initial vector length, in bytes. + */ + public int ivLength() { + return conf.getInt("spark.network.crypto.ivLength", 16); + } + + /** + * The algorithm for generated secret keys. Nobody should really need to change this, + * but configurable just in case. + */ + public String keyAlgorithm() { + return conf.get("spark.network.crypto.keyAlgorithm", "AES"); + } + + /** + * Whether to fall back to SASL if the new auth protocol fails. Enabled by default for + * backwards compatibility. + */ + public boolean saslFallback() { + return conf.getBoolean("spark.network.crypto.saslFallback", true); } /** - * The trigger for enabling AES encryption. + * Whether to enable SASL-based encryption when authenticating using SASL. */ - public boolean aesEncryptionEnabled() { - return conf.getBoolean("spark.network.aes.enabled", false); + public boolean saslEncryption() { + return conf.getBoolean("spark.authenticate.enableSaslEncryption", false); } /** - * The key size to use when AES cipher is enabled. Notice that the length should be 16, 24 or 32 - * bytes. + * Maximum number of bytes to be encrypted at a time when SASL encryption is used. */ - public int aesCipherKeySize() { - return conf.getInt("spark.network.aes.keySize", 16); + public int maxSaslEncryptedBlockSize() { + return Ints.checkedCast(JavaUtils.byteStringAsBytes( + conf.get("spark.network.sasl.maxEncryptedBlockSize", "64k"))); + } + + /** + * Whether the server should enforce encryption on SASL-authenticated connections. + */ + public boolean saslServerAlwaysEncrypt() { + return conf.getBoolean("spark.network.sasl.serverAlwaysEncrypt", false); } /** * The commons-crypto configuration for the module. */ public Properties cryptoConf() { - return CryptoUtils.toCryptoConf("spark.network.aes.config.", conf.getAll()); + return CryptoUtils.toCryptoConf("spark.network.crypto.config.", conf.getAll()); } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java new file mode 100644 index 0000000000000..9a186f211312f --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.util.Arrays; +import java.util.Map; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.collect.ImmutableMap; +import org.junit.BeforeClass; +import org.junit.Test; +import static org.junit.Assert.*; + +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class AuthEngineSuite { + + private static TransportConf conf; + + @BeforeClass + public static void setUp() { + conf = new TransportConf("rpc", MapConfigProvider.EMPTY); + } + + @Test + public void testAuthEngine() throws Exception { + AuthEngine client = new AuthEngine("appId", "secret", conf); + AuthEngine server = new AuthEngine("appId", "secret", conf); + + try { + ClientChallenge clientChallenge = client.challenge(); + ServerResponse serverResponse = server.respond(clientChallenge); + client.validate(serverResponse); + + TransportCipher serverCipher = server.sessionCipher(); + TransportCipher clientCipher = client.sessionCipher(); + + assertTrue(Arrays.equals(serverCipher.getInputIv(), clientCipher.getOutputIv())); + assertTrue(Arrays.equals(serverCipher.getOutputIv(), clientCipher.getInputIv())); + assertEquals(serverCipher.getKey(), clientCipher.getKey()); + } finally { + client.close(); + server.close(); + } + } + + @Test + public void testMismatchedSecret() throws Exception { + AuthEngine client = new AuthEngine("appId", "secret", conf); + AuthEngine server = new AuthEngine("appId", "different_secret", conf); + + ClientChallenge clientChallenge = client.challenge(); + try { + server.respond(clientChallenge); + fail("Should have failed to validate response."); + } catch (IllegalArgumentException e) { + // Expected. + } + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongAppId() throws Exception { + AuthEngine engine = new AuthEngine("appId", "secret", conf); + ClientChallenge challenge = engine.challenge(); + + byte[] badChallenge = engine.challenge(new byte[] { 0x00 }, challenge.nonce, + engine.rawResponse(engine.challenge)); + engine.respond(new ClientChallenge(challenge.appId, challenge.kdf, challenge.iterations, + challenge.cipher, challenge.keyLength, challenge.nonce, badChallenge)); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongNonce() throws Exception { + AuthEngine engine = new AuthEngine("appId", "secret", conf); + ClientChallenge challenge = engine.challenge(); + + byte[] badChallenge = engine.challenge(challenge.appId.getBytes(UTF_8), new byte[] { 0x00 }, + engine.rawResponse(engine.challenge)); + engine.respond(new ClientChallenge(challenge.appId, challenge.kdf, challenge.iterations, + challenge.cipher, challenge.keyLength, challenge.nonce, badChallenge)); + } + + @Test(expected = IllegalArgumentException.class) + public void testBadChallenge() throws Exception { + AuthEngine engine = new AuthEngine("appId", "secret", conf); + ClientChallenge challenge = engine.challenge(); + + byte[] badChallenge = new byte[challenge.challenge.length]; + engine.respond(new ClientChallenge(challenge.appId, challenge.kdf, challenge.iterations, + challenge.cipher, challenge.keyLength, challenge.nonce, badChallenge)); + } + +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java new file mode 100644 index 0000000000000..21609d5aa2a20 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java @@ -0,0 +1,213 @@ +/* + * 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.network.crypto; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.netty.channel.Channel; +import org.junit.After; +import org.junit.Test; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.sasl.SaslRpcHandler; +import org.apache.spark.network.sasl.SaslServerBootstrap; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class AuthIntegrationSuite { + + private AuthTestCtx ctx; + + @After + public void cleanUp() throws Exception { + if (ctx != null) { + ctx.close(); + } + ctx = null; + } + + @Test + public void testNewAuth() throws Exception { + ctx = new AuthTestCtx(); + ctx.createServer("secret"); + ctx.createClient("secret"); + + ByteBuffer reply = ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + assertEquals("Pong", JavaUtils.bytesToString(reply)); + assertTrue(ctx.authRpcHandler.doDelegate); + assertFalse(ctx.authRpcHandler.delegate instanceof SaslRpcHandler); + } + + @Test + public void testAuthFailure() throws Exception { + ctx = new AuthTestCtx(); + ctx.createServer("server"); + + try { + ctx.createClient("client"); + fail("Should have failed to create client."); + } catch (Exception e) { + assertFalse(ctx.authRpcHandler.doDelegate); + assertFalse(ctx.serverChannel.isActive()); + } + } + + @Test + public void testSaslServerFallback() throws Exception { + ctx = new AuthTestCtx(); + ctx.createServer("secret", true); + ctx.createClient("secret", false); + + ByteBuffer reply = ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + assertEquals("Pong", JavaUtils.bytesToString(reply)); + } + + @Test + public void testSaslClientFallback() throws Exception { + ctx = new AuthTestCtx(); + ctx.createServer("secret", false); + ctx.createClient("secret", true); + + ByteBuffer reply = ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + assertEquals("Pong", JavaUtils.bytesToString(reply)); + } + + @Test + public void testAuthReplay() throws Exception { + // This test covers the case where an attacker replays a challenge message sniffed from the + // network, but doesn't know the actual secret. The server should close the connection as + // soon as a message is sent after authentication is performed. This is emulated by removing + // the client encryption handler after authentication. + ctx = new AuthTestCtx(); + ctx.createServer("secret"); + ctx.createClient("secret"); + + assertNotNull(ctx.client.getChannel().pipeline() + .remove(TransportCipher.ENCRYPTION_HANDLER_NAME)); + + try { + ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + fail("Should have failed unencrypted RPC."); + } catch (Exception e) { + assertTrue(ctx.authRpcHandler.doDelegate); + } + } + + private class AuthTestCtx { + + private final String appId = "testAppId"; + private final TransportConf conf; + private final TransportContext ctx; + + TransportClient client; + TransportServer server; + volatile Channel serverChannel; + volatile AuthRpcHandler authRpcHandler; + + AuthTestCtx() throws Exception { + Map testConf = ImmutableMap.of("spark.network.crypto.enabled", "true"); + this.conf = new TransportConf("rpc", new MapConfigProvider(testConf)); + + RpcHandler rpcHandler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + assertEquals("Ping", JavaUtils.bytesToString(message)); + callback.onSuccess(JavaUtils.stringToBytes("Pong")); + } + + @Override + public StreamManager getStreamManager() { + return null; + } + }; + + this.ctx = new TransportContext(conf, rpcHandler); + } + + void createServer(String secret) throws Exception { + createServer(secret, true); + } + + void createServer(String secret, boolean enableAes) throws Exception { + TransportServerBootstrap introspector = new TransportServerBootstrap() { + @Override + public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { + AuthTestCtx.this.serverChannel = channel; + if (rpcHandler instanceof AuthRpcHandler) { + AuthTestCtx.this.authRpcHandler = (AuthRpcHandler) rpcHandler; + } + return rpcHandler; + } + }; + SecretKeyHolder keyHolder = createKeyHolder(secret); + TransportServerBootstrap auth = enableAes ? new AuthServerBootstrap(conf, keyHolder) + : new SaslServerBootstrap(conf, keyHolder); + this.server = ctx.createServer(Lists.newArrayList(auth, introspector)); + } + + void createClient(String secret) throws Exception { + createClient(secret, true); + } + + void createClient(String secret, boolean enableAes) throws Exception { + TransportConf clientConf = enableAes ? conf + : new TransportConf("rpc", MapConfigProvider.EMPTY); + List bootstraps = Lists.newArrayList( + new AuthClientBootstrap(clientConf, appId, createKeyHolder(secret))); + this.client = ctx.createClientFactory(bootstraps) + .createClient(TestUtils.getLocalHost(), server.getPort()); + } + + void close() { + if (client != null) { + client.close(); + } + if (server != null) { + server.close(); + } + } + + private SecretKeyHolder createKeyHolder(String secret) { + SecretKeyHolder keyHolder = mock(SecretKeyHolder.class); + when(keyHolder.getSaslUser(anyString())).thenReturn(appId); + when(keyHolder.getSecretKey(anyString())).thenReturn(secret); + return keyHolder; + } + + } + +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthMessagesSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthMessagesSuite.java new file mode 100644 index 0000000000000..a90ff247da4fc --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthMessagesSuite.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.nio.ByteBuffer; +import java.util.Arrays; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.junit.Test; +import static org.junit.Assert.*; + +import org.apache.spark.network.protocol.Encodable; + +public class AuthMessagesSuite { + + private static int COUNTER = 0; + + private static String string() { + return String.valueOf(COUNTER++); + } + + private static byte[] byteArray() { + byte[] bytes = new byte[COUNTER++]; + for (int i = 0; i < bytes.length; i++) { + bytes[i] = (byte) COUNTER; + } return bytes; + } + + private static int integer() { + return COUNTER++; + } + + @Test + public void testClientChallenge() { + ClientChallenge msg = new ClientChallenge(string(), string(), integer(), string(), integer(), + byteArray(), byteArray()); + ClientChallenge decoded = ClientChallenge.decodeMessage(encode(msg)); + + assertEquals(msg.appId, decoded.appId); + assertEquals(msg.kdf, decoded.kdf); + assertEquals(msg.iterations, decoded.iterations); + assertEquals(msg.cipher, decoded.cipher); + assertEquals(msg.keyLength, decoded.keyLength); + assertTrue(Arrays.equals(msg.nonce, decoded.nonce)); + assertTrue(Arrays.equals(msg.challenge, decoded.challenge)); + } + + @Test + public void testServerResponse() { + ServerResponse msg = new ServerResponse(byteArray(), byteArray(), byteArray(), byteArray()); + ServerResponse decoded = ServerResponse.decodeMessage(encode(msg)); + assertTrue(Arrays.equals(msg.response, decoded.response)); + assertTrue(Arrays.equals(msg.nonce, decoded.nonce)); + assertTrue(Arrays.equals(msg.inputIv, decoded.inputIv)); + assertTrue(Arrays.equals(msg.outputIv, decoded.outputIv)); + } + + private ByteBuffer encode(Encodable msg) { + ByteBuf buf = Unpooled.buffer(); + msg.encode(buf); + return buf.nioBuffer(); + } + +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index e27301f49e34b..87129b900bf0b 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -56,7 +56,6 @@ import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientBootstrap; -import org.apache.spark.network.sasl.aes.AesCipher; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.server.TransportServer; @@ -153,7 +152,7 @@ public Void answer(InvocationOnMock invocation) { .when(rpcHandler) .receive(any(TransportClient.class), any(ByteBuffer.class), any(RpcResponseCallback.class)); - SaslTestCtx ctx = new SaslTestCtx(rpcHandler, encrypt, false, false); + SaslTestCtx ctx = new SaslTestCtx(rpcHandler, encrypt, false); try { ByteBuffer response = ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), TimeUnit.SECONDS.toMillis(10)); @@ -279,7 +278,7 @@ public ManagedBuffer answer(InvocationOnMock invocation) { new Random().nextBytes(data); Files.write(data, file); - ctx = new SaslTestCtx(rpcHandler, true, false, false, testConf); + ctx = new SaslTestCtx(rpcHandler, true, false, testConf); final CountDownLatch lock = new CountDownLatch(1); @@ -317,7 +316,7 @@ public Void answer(InvocationOnMock invocation) { public void testServerAlwaysEncrypt() throws Exception { SaslTestCtx ctx = null; try { - ctx = new SaslTestCtx(mock(RpcHandler.class), false, false, false, + ctx = new SaslTestCtx(mock(RpcHandler.class), false, false, ImmutableMap.of("spark.network.sasl.serverAlwaysEncrypt", "true")); fail("Should have failed to connect without encryption."); } catch (Exception e) { @@ -336,7 +335,7 @@ public void testDataEncryptionIsActuallyEnabled() throws Exception { // able to understand RPCs sent to it and thus close the connection. SaslTestCtx ctx = null; try { - ctx = new SaslTestCtx(mock(RpcHandler.class), true, true, false); + ctx = new SaslTestCtx(mock(RpcHandler.class), true, true); ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), TimeUnit.SECONDS.toMillis(10)); fail("Should have failed to send RPC to server."); @@ -374,69 +373,6 @@ public void testDelegates() throws Exception { } } - @Test - public void testAesEncryption() throws Exception { - final AtomicReference response = new AtomicReference<>(); - final File file = File.createTempFile("sasltest", ".txt"); - SaslTestCtx ctx = null; - try { - final TransportConf conf = new TransportConf("rpc", MapConfigProvider.EMPTY); - final TransportConf spyConf = spy(conf); - doReturn(true).when(spyConf).aesEncryptionEnabled(); - - StreamManager sm = mock(StreamManager.class); - when(sm.getChunk(anyLong(), anyInt())).thenAnswer(new Answer() { - @Override - public ManagedBuffer answer(InvocationOnMock invocation) { - return new FileSegmentManagedBuffer(spyConf, file, 0, file.length()); - } - }); - - RpcHandler rpcHandler = mock(RpcHandler.class); - when(rpcHandler.getStreamManager()).thenReturn(sm); - - byte[] data = new byte[256 * 1024 * 1024]; - new Random().nextBytes(data); - Files.write(data, file); - - ctx = new SaslTestCtx(rpcHandler, true, false, true); - - final Object lock = new Object(); - - ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) { - response.set((ManagedBuffer) invocation.getArguments()[1]); - response.get().retain(); - synchronized (lock) { - lock.notifyAll(); - } - return null; - } - }).when(callback).onSuccess(anyInt(), any(ManagedBuffer.class)); - - synchronized (lock) { - ctx.client.fetchChunk(0, 0, callback); - lock.wait(10 * 1000); - } - - verify(callback, times(1)).onSuccess(anyInt(), any(ManagedBuffer.class)); - verify(callback, never()).onFailure(anyInt(), any(Throwable.class)); - - byte[] received = ByteStreams.toByteArray(response.get().createInputStream()); - assertTrue(Arrays.equals(data, received)); - } finally { - file.delete(); - if (ctx != null) { - ctx.close(); - } - if (response.get() != null) { - response.get().release(); - } - } - } - private static class SaslTestCtx { final TransportClient client; @@ -449,46 +385,39 @@ private static class SaslTestCtx { SaslTestCtx( RpcHandler rpcHandler, boolean encrypt, - boolean disableClientEncryption, - boolean aesEnable) + boolean disableClientEncryption) throws Exception { - this(rpcHandler, encrypt, disableClientEncryption, aesEnable, - Collections.emptyMap()); + this(rpcHandler, encrypt, disableClientEncryption, Collections.emptyMap()); } SaslTestCtx( RpcHandler rpcHandler, boolean encrypt, boolean disableClientEncryption, - boolean aesEnable, - Map testConf) + Map extraConf) throws Exception { + Map testConf = ImmutableMap.builder() + .putAll(extraConf) + .put("spark.authenticate.enableSaslEncryption", String.valueOf(encrypt)) + .build(); TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(testConf)); - if (aesEnable) { - conf = spy(conf); - doReturn(true).when(conf).aesEncryptionEnabled(); - } - SecretKeyHolder keyHolder = mock(SecretKeyHolder.class); when(keyHolder.getSaslUser(anyString())).thenReturn("user"); when(keyHolder.getSecretKey(anyString())).thenReturn("secret"); TransportContext ctx = new TransportContext(conf, rpcHandler); - String encryptHandlerName = aesEnable ? AesCipher.ENCRYPTION_HANDLER_NAME : - SaslEncryption.ENCRYPTION_HANDLER_NAME; - - this.checker = new EncryptionCheckerBootstrap(encryptHandlerName); + this.checker = new EncryptionCheckerBootstrap(SaslEncryption.ENCRYPTION_HANDLER_NAME); this.server = ctx.createServer(Arrays.asList(new SaslServerBootstrap(conf, keyHolder), checker)); try { List clientBootstraps = Lists.newArrayList(); - clientBootstraps.add(new SaslClientBootstrap(conf, "user", keyHolder, encrypt)); + clientBootstraps.add(new SaslClientBootstrap(conf, "user", keyHolder)); if (disableClientEncryption) { clientBootstraps.add(new EncryptionDisablerBootstrap()); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index 772fb88325b35..616505d9796d0 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -21,7 +21,6 @@ import java.nio.ByteBuffer; import java.util.List; -import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,7 +29,7 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientBootstrap; import org.apache.spark.network.client.TransportClientFactory; -import org.apache.spark.network.sasl.SaslClientBootstrap; +import org.apache.spark.network.crypto.AuthClientBootstrap; import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; @@ -47,8 +46,7 @@ public class ExternalShuffleClient extends ShuffleClient { private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleClient.class); private final TransportConf conf; - private final boolean saslEnabled; - private final boolean saslEncryptionEnabled; + private final boolean authEnabled; private final SecretKeyHolder secretKeyHolder; protected TransportClientFactory clientFactory; @@ -61,15 +59,10 @@ public class ExternalShuffleClient extends ShuffleClient { public ExternalShuffleClient( TransportConf conf, SecretKeyHolder secretKeyHolder, - boolean saslEnabled, - boolean saslEncryptionEnabled) { - Preconditions.checkArgument( - !saslEncryptionEnabled || saslEnabled, - "SASL encryption can only be enabled if SASL is also enabled."); + boolean authEnabled) { this.conf = conf; this.secretKeyHolder = secretKeyHolder; - this.saslEnabled = saslEnabled; - this.saslEncryptionEnabled = saslEncryptionEnabled; + this.authEnabled = authEnabled; } protected void checkInit() { @@ -81,8 +74,8 @@ public void init(String appId) { this.appId = appId; TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true); List bootstraps = Lists.newArrayList(); - if (saslEnabled) { - bootstraps.add(new SaslClientBootstrap(conf, appId, secretKeyHolder, saslEncryptionEnabled)); + if (authEnabled) { + bootstraps.add(new AuthClientBootstrap(conf, appId, secretKeyHolder)); } clientFactory = context.createClientFactory(bootstraps); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java index 42cedd9943150..ab49b1c1d7896 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java @@ -60,9 +60,8 @@ public class MesosExternalShuffleClient extends ExternalShuffleClient { public MesosExternalShuffleClient( TransportConf conf, SecretKeyHolder secretKeyHolder, - boolean saslEnabled, - boolean saslEncryptionEnabled) { - super(conf, secretKeyHolder, saslEnabled, saslEncryptionEnabled); + boolean authEnabled) { + super(conf, secretKeyHolder, authEnabled); } public void registerDriverWithShuffleService( diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 8dd97b29eb368..9248ef3c467df 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -133,7 +133,7 @@ private FetchResult fetchBlocks( final Semaphore requestsRemaining = new Semaphore(0); - ExternalShuffleClient client = new ExternalShuffleClient(clientConf, null, false, false); + ExternalShuffleClient client = new ExternalShuffleClient(clientConf, null, false); client.init(APP_ID); client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, new BlockFetchingListener() { @@ -243,7 +243,7 @@ public void testFetchNoServer() throws Exception { private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) throws IOException { - ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false, false); + ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false); client.init(APP_ID); client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), executorId, executorInfo); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index aed25a161e17e..4ae75a1b1762a 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Arrays; +import com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -88,8 +89,14 @@ public void testEncryption() throws IOException { /** Creates an ExternalShuffleClient and attempts to register with the server. */ private void validate(String appId, String secretKey, boolean encrypt) throws IOException { + TransportConf testConf = conf; + if (encrypt) { + testConf = new TransportConf("shuffle", new MapConfigProvider( + ImmutableMap.of("spark.authenticate.enableSaslEncryption", "true"))); + } + ExternalShuffleClient client = - new ExternalShuffleClient(conf, new TestSecretKeyHolder(appId, secretKey), true, encrypt); + new ExternalShuffleClient(testConf, new TestSecretKeyHolder(appId, secretKey), true); client.init(appId); // Registration either succeeds or throws an exception. client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), "exec0", diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index ea726e3c8240e..c7620d0fe1288 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -45,7 +45,7 @@ import org.slf4j.LoggerFactory; import org.apache.spark.network.TransportContext; -import org.apache.spark.network.sasl.SaslServerBootstrap; +import org.apache.spark.network.crypto.AuthServerBootstrap; import org.apache.spark.network.sasl.ShuffleSecretManager; import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; @@ -172,7 +172,7 @@ protected void serviceInit(Configuration conf) throws Exception { boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); if (authEnabled) { createSecretManager(); - bootstraps.add(new SaslServerBootstrap(transportConf, secretManager)); + bootstraps.add(new AuthServerBootstrap(transportConf, secretManager)); } int port = conf.getInt( diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index 97eed611e8f9a..9fe97b4d9c20c 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -113,6 +113,26 @@ public final void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved onEvent(executorRemoved); } + @Override + public final void onExecutorBlacklisted(SparkListenerExecutorBlacklisted executorBlacklisted) { + onEvent(executorBlacklisted); + } + + @Override + public final void onExecutorUnblacklisted(SparkListenerExecutorUnblacklisted executorUnblacklisted) { + onEvent(executorUnblacklisted); + } + + @Override + public final void onNodeBlacklisted(SparkListenerNodeBlacklisted nodeBlacklisted) { + onEvent(nodeBlacklisted); + } + + @Override + public final void onNodeUnblacklisted(SparkListenerNodeUnblacklisted nodeUnblacklisted) { + onEvent(nodeUnblacklisted); + } + @Override public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { onEvent(blockUpdated); 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 89a7963a86d98..277010015072a 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -36,3 +36,7 @@ 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 + +# Parquet related logging +log4j.logger.org.apache.parquet.CorruptStatistics=ERROR +log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 64ea719141f4b..4e83d6d564986 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -45,6 +45,11 @@

Summary

title="Bytes and records written to disk in order to be read by a shuffle in a future stage."> Shuffle Write + + + Blacklisted + diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index fe5db6aa26b65..7dbfe32de903a 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -182,7 +182,7 @@ $(document).ready(function () { executorsSummary = $("#active-executors"); getStandAloneppId(function (appId) { - + var endPoint = createRESTEndPoint(appId); $.getJSON(endPoint, function (response, status, jqXHR) { var summary = []; @@ -202,7 +202,8 @@ $(document).ready(function () { var allTotalInputBytes = 0; var allTotalShuffleRead = 0; var allTotalShuffleWrite = 0; - + var allTotalBlacklisted = 0; + var activeExecCnt = 0; var activeRDDBlocks = 0; var activeMemoryUsed = 0; @@ -219,7 +220,8 @@ $(document).ready(function () { var activeTotalInputBytes = 0; var activeTotalShuffleRead = 0; var activeTotalShuffleWrite = 0; - + var activeTotalBlacklisted = 0; + var deadExecCnt = 0; var deadRDDBlocks = 0; var deadMemoryUsed = 0; @@ -236,7 +238,8 @@ $(document).ready(function () { var deadTotalInputBytes = 0; var deadTotalShuffleRead = 0; var deadTotalShuffleWrite = 0; - + var deadTotalBlacklisted = 0; + response.forEach(function (exec) { allExecCnt += 1; allRDDBlocks += exec.rddBlocks; @@ -254,6 +257,7 @@ $(document).ready(function () { allTotalInputBytes += exec.totalInputBytes; allTotalShuffleRead += exec.totalShuffleRead; allTotalShuffleWrite += exec.totalShuffleWrite; + allTotalBlacklisted += exec.isBlacklisted ? 1 : 0; if (exec.isActive) { activeExecCnt += 1; activeRDDBlocks += exec.rddBlocks; @@ -271,6 +275,7 @@ $(document).ready(function () { activeTotalInputBytes += exec.totalInputBytes; activeTotalShuffleRead += exec.totalShuffleRead; activeTotalShuffleWrite += exec.totalShuffleWrite; + activeTotalBlacklisted += exec.isBlacklisted ? 1 : 0; } else { deadExecCnt += 1; deadRDDBlocks += exec.rddBlocks; @@ -288,9 +293,10 @@ $(document).ready(function () { deadTotalInputBytes += exec.totalInputBytes; deadTotalShuffleRead += exec.totalShuffleRead; deadTotalShuffleWrite += exec.totalShuffleWrite; + deadTotalBlacklisted += exec.isBlacklisted ? 1 : 0; } }); - + var totalSummary = { "execCnt": ( "Total(" + allExecCnt + ")"), "allRDDBlocks": allRDDBlocks, @@ -307,7 +313,8 @@ $(document).ready(function () { "allTotalGCTime": allTotalGCTime, "allTotalInputBytes": allTotalInputBytes, "allTotalShuffleRead": allTotalShuffleRead, - "allTotalShuffleWrite": allTotalShuffleWrite + "allTotalShuffleWrite": allTotalShuffleWrite, + "allTotalBlacklisted": allTotalBlacklisted }; var activeSummary = { "execCnt": ( "Active(" + activeExecCnt + ")"), @@ -325,7 +332,8 @@ $(document).ready(function () { "allTotalGCTime": activeTotalGCTime, "allTotalInputBytes": activeTotalInputBytes, "allTotalShuffleRead": activeTotalShuffleRead, - "allTotalShuffleWrite": activeTotalShuffleWrite + "allTotalShuffleWrite": activeTotalShuffleWrite, + "allTotalBlacklisted": activeTotalBlacklisted }; var deadSummary = { "execCnt": ( "Dead(" + deadExecCnt + ")" ), @@ -343,12 +351,13 @@ $(document).ready(function () { "allTotalGCTime": deadTotalGCTime, "allTotalInputBytes": deadTotalInputBytes, "allTotalShuffleRead": deadTotalShuffleRead, - "allTotalShuffleWrite": deadTotalShuffleWrite + "allTotalShuffleWrite": deadTotalShuffleWrite, + "allTotalBlacklisted": deadTotalBlacklisted }; - + var data = {executors: response, "execSummary": [activeSummary, deadSummary, totalSummary]}; $.get(createTemplateURI(appId), function (template) { - + executorsSummary.append(Mustache.render($(template).filter("#executors-summary-template").html(), data)); var selector = "#active-executors-table"; var conf = { @@ -360,7 +369,12 @@ $(document).ready(function () { } }, {data: 'hostPort'}, - {data: 'isActive', render: formatStatus}, + {data: 'isActive', render: function (data, type, row) { + if (type !== 'display') return data; + if (row.isBlacklisted) return "Blacklisted"; + else return formatStatus (data, type); + } + }, {data: 'rddBlocks'}, { data: function (row, type) { @@ -474,7 +488,8 @@ $(document).ready(function () { }, {data: 'allTotalInputBytes', render: formatBytes}, {data: 'allTotalShuffleRead', render: formatBytes}, - {data: 'allTotalShuffleWrite', render: formatBytes} + {data: 'allTotalShuffleWrite', render: formatBytes}, + {data: 'allTotalBlacklisted'} ], "paging": false, "searching": false, diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js index 1b0d4692d9cd0..75b959fdeb59a 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js @@ -35,7 +35,7 @@ * primitives (e.g. take, any SQL query). * * In the visualization, an RDD is expressed as a node, and its dependencies - * as directed edges (from parent to child). operation scopes, stages, and + * as directed edges (from parent to child). Operation scopes, stages, and * jobs are expressed as clusters that may contain one or many nodes. These * clusters may be nested inside of each other in the scenarios described * above. @@ -173,6 +173,7 @@ function renderDagViz(forJob) { }); resizeSvg(svg); + interpretLineBreak(svg); } /* Render the RDD DAG visualization on the stage page. */ @@ -362,6 +363,27 @@ function resizeSvg(svg) { .attr("height", height); } +/* + * Helper function to interpret line break for tag 'tspan'. + * For tag 'tspan', line break '/n' is display in UI as raw for both stage page and job page, + * here this function is to enable line break. + */ +function interpretLineBreak(svg) { + var allTSpan = svg.selectAll("tspan").each(function() { + node = d3.select(this); + var original = node[0][0].innerHTML; + if (original.indexOf("\\n") != -1) { + var arr = original.split("\\n"); + var newNode = this.cloneNode(this); + + node[0][0].innerHTML = arr[0]; + newNode.innerHTML = arr[1]; + + this.parentNode.appendChild(newNode); + } + }); +} + /* * (Job page only) Helper function to draw edges that cross stage boundaries. * We need to do this manually because we render each stage separately in dagre-d3. @@ -470,15 +492,23 @@ function connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer) { edgesContainer.append("path").datum(points).attr("d", line); } +/* + * Replace `/n` with `
` + */ +function replaceLineBreak(str) { + return str.replace("\\n", "
"); +} + /* (Job page only) Helper function to add tooltips for RDDs. */ function addTooltipsForRDDs(svgContainer) { svgContainer.selectAll("g.node").each(function() { var node = d3.select(this); - var tooltipText = node.attr("name"); + var tooltipText = replaceLineBreak(node.attr("name")); if (tooltipText) { node.select("circle") .attr("data-toggle", "tooltip") .attr("data-placement", "bottom") + .attr("data-html", "true") // to interpret line break, tooltipText is showing title .attr("title", tooltipText); } // Link tooltips for all nodes that belong to the same RDD diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala index 7bea636c94aa0..9d5fbefc824ad 100644 --- a/core/src/main/scala/org/apache/spark/Accumulator.scala +++ b/core/src/main/scala/org/apache/spark/Accumulator.scala @@ -24,8 +24,7 @@ package org.apache.spark * They can be used to implement counters (as in MapReduce) or sums. Spark natively supports * accumulators of numeric value types, and programmers can add support for new types. * - * An accumulator is created from an initial value `v` by calling - * [[SparkContext#accumulator SparkContext.accumulator]]. + * An accumulator is created from an initial value `v` by calling `SparkContext.accumulator`. * Tasks running on the cluster can then add to it using the `+=` operator. * However, they cannot read its value. Only the driver program can read the accumulator's value, * using its [[#value]] method. diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 87fe56315203e..cde768281f119 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -26,7 +26,6 @@ import javax.net.ssl._ import com.google.common.hash.HashCodes import com.google.common.io.Files import org.apache.hadoop.io.Text -import org.apache.hadoop.security.Credentials import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging @@ -193,7 +192,7 @@ private[spark] class SecurityManager( // allow all users/groups to have view/modify permissions private val WILDCARD_ACL = "*" - private val authOn = sparkConf.getBoolean(SecurityManager.SPARK_AUTH_CONF, false) + private val authOn = sparkConf.get(NETWORK_AUTH_ENABLED) // keep spark.ui.acls.enable for backwards compatibility with 1.0 private var aclsOn = sparkConf.getBoolean("spark.acls.enable", sparkConf.getBoolean("spark.ui.acls.enable", false)) @@ -518,11 +517,11 @@ private[spark] class SecurityManager( def isAuthenticationEnabled(): Boolean = authOn /** - * Checks whether SASL encryption should be enabled. - * @return Whether to enable SASL encryption when connecting to services that support it. + * Checks whether network encryption should be enabled. + * @return Whether to enable encryption when connecting to services that support it. */ - def isSaslEncryptionEnabled(): Boolean = { - sparkConf.getBoolean("spark.authenticate.enableSaslEncryption", false) + def isEncryptionEnabled(): Boolean = { + sparkConf.get(NETWORK_ENCRYPTION_ENABLED) || sparkConf.get(SASL_ENCRYPTION_ENABLED) } /** diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index d78b9f1b29685..308a1ed5fa963 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -607,6 +607,10 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria "\"client\".") } } + + val encryptionEnabled = get(NETWORK_ENCRYPTION_ENABLED) || get(SASL_ENCRYPTION_ENABLED) + require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED), + s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.") } /** @@ -699,8 +703,10 @@ private[spark] object SparkConf extends Logging { "spark.rpc.message.maxSize" -> Seq( AlternateConfig("spark.akka.frameSize", "1.6")), "spark.yarn.jars" -> Seq( - AlternateConfig("spark.yarn.jar", "2.0")) - ) + AlternateConfig("spark.yarn.jar", "2.0")), + "spark.yarn.access.hadoopFileSystems" -> Seq( + AlternateConfig("spark.yarn.access.namenodes", "2.2")) + ) /** * A view of `configsWithAlternatives` that makes it more efficient to look up deprecated @@ -724,6 +730,7 @@ private[spark] object SparkConf extends Logging { (name.startsWith("spark.auth") && name != SecurityManager.SPARK_AUTH_SECRET_CONF) || name.startsWith("spark.ssl") || name.startsWith("spark.rpc") || + name.startsWith("spark.network") || isSparkPortConf(name) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index efb5f9d501e48..869c5d7094cd4 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -19,7 +19,7 @@ package org.apache.spark import java.io._ import java.lang.reflect.Constructor -import java.net.{URI} +import java.net.URI import java.util.{Arrays, Locale, Properties, ServiceLoader, UUID} import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference} diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 1296386ac9bd3..539dbb55eeff0 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -235,7 +235,7 @@ object SparkEnv extends Logging { val securityManager = new SecurityManager(conf, ioEncryptionKey) ioEncryptionKey.foreach { _ => - if (!securityManager.isSaslEncryptionEnabled()) { + if (!securityManager.isEncryptionEnabled()) { logWarning("I/O encryption enabled without RPC encryption: keys will be visible on the " + "wire.") } diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index fd0477541ef03..109104f0a537b 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -18,11 +18,15 @@ package org.apache.spark import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} -import java.net.{URI, URL} +import java.net.{HttpURLConnection, URI, URL} import java.nio.charset.StandardCharsets +import java.security.SecureRandom +import java.security.cert.X509Certificate import java.util.Arrays import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.jar.{JarEntry, JarOutputStream} +import javax.net.ssl._ +import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -31,7 +35,6 @@ import scala.sys.process.{Process, ProcessLogger} import scala.util.Try import com.google.common.io.{ByteStreams, Files} -import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ @@ -194,6 +197,37 @@ private[spark] object TestUtils { attempt.isSuccess && attempt.get == 0 } + /** + * Returns the response code from an HTTP(S) URL. + */ + def httpResponseCode(url: URL, method: String = "GET"): Int = { + val connection = url.openConnection().asInstanceOf[HttpURLConnection] + connection.setRequestMethod(method) + + // Disable cert and host name validation for HTTPS tests. + if (connection.isInstanceOf[HttpsURLConnection]) { + val sslCtx = SSLContext.getInstance("SSL") + val trustManager = new X509TrustManager { + override def getAcceptedIssuers(): Array[X509Certificate] = null + override def checkClientTrusted(x509Certificates: Array[X509Certificate], s: String) {} + override def checkServerTrusted(x509Certificates: Array[X509Certificate], s: String) {} + } + val verifier = new HostnameVerifier() { + override def verify(hostname: String, session: SSLSession): Boolean = true + } + sslCtx.init(null, Array(trustManager), new SecureRandom()) + connection.asInstanceOf[HttpsURLConnection].setSSLSocketFactory(sslCtx.getSocketFactory()) + connection.asInstanceOf[HttpsURLConnection].setHostnameVerifier(verifier) + } + + try { + connection.connect() + connection.getResponseCode() + } finally { + connection.disconnect() + } + } + } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 766aea213a972..9544475ff0428 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -166,7 +166,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). * - * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * This method differs from `sampleByKey` in that we make additional passes over the RDD to * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) * over all key values with a 99.99% confidence. When sampling without replacement, we need one * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need @@ -184,7 +184,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). * - * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * This method differs from `sampleByKey` in that we make additional passes over the RDD to * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) * over all key values with a 99.99% confidence. When sampling without replacement, we need one * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need 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 eda16d957cc58..91ae1002abd21 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 @@ -393,7 +393,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def treeReduce(f: JFunction2[T, T, T], depth: Int): T = rdd.treeReduce(f, depth) /** - * [[org.apache.spark.api.java.JavaRDDLike#treeReduce]] with suggested depth 2. + * `org.apache.spark.api.java.JavaRDDLike.treeReduce` with suggested depth 2. */ def treeReduce(f: JFunction2[T, T, T]): T = treeReduce(f, 2) @@ -440,7 +440,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * [[org.apache.spark.api.java.JavaRDDLike#treeAggregate]] with suggested depth 2. + * `org.apache.spark.api.java.JavaRDDLike.treeAggregate` with suggested depth 2. */ def treeAggregate[U]( zeroValue: U, diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index 13eadbe44f612..8d491ddf6e092 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -25,8 +25,8 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.TransportContext +import org.apache.spark.network.crypto.AuthServerBootstrap import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.sasl.SaslServerBootstrap import org.apache.spark.network.server.{TransportServer, TransportServerBootstrap} import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler import org.apache.spark.network.util.TransportConf @@ -47,7 +47,6 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana private val enabled = sparkConf.getBoolean("spark.shuffle.service.enabled", false) private val port = sparkConf.getInt("spark.shuffle.service.port", 7337) - private val useSasl: Boolean = securityManager.isAuthenticationEnabled() private val transportConf = SparkTransportConf.fromSparkConf(sparkConf, "shuffle", numUsableCores = 0) @@ -74,10 +73,11 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana /** Start the external shuffle service */ def start() { require(server == null, "Shuffle server already started") - logInfo(s"Starting shuffle service on port $port with useSasl = $useSasl") + val authEnabled = securityManager.isAuthenticationEnabled() + logInfo(s"Starting shuffle service on port $port (auth enabled = $authEnabled)") val bootstraps: Seq[TransportServerBootstrap] = - if (useSasl) { - Seq(new SaslServerBootstrap(transportConf, securityManager)) + if (authEnabled) { + Seq(new AuthServerBootstrap(transportConf, securityManager)) } else { Nil } diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleServiceSource.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleServiceSource.scala index e917679c83877..357a9769311a9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleServiceSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleServiceSource.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy import javax.annotation.concurrent.ThreadSafe -import com.codahale.metrics.{Gauge, MetricRegistry} +import com.codahale.metrics.MetricRegistry import org.apache.spark.metrics.source.Source import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index a980144a75953..5ffdedd1658ab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import java.io.{File, IOException, PrintStream} +import java.io.{File, IOException} import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} import java.net.URL import java.security.PrivilegedExceptionAction diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index b1d36e1821cc7..dee77343d806d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -506,7 +506,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to place | on the PYTHONPATH for Python apps. | --files FILES Comma-separated list of files to be placed in the working - | directory of each executor. + | directory of each executor. File paths of these files + | in executors can be accessed via SparkFiles.get(fileName). | | --conf PROP=VALUE Arbitrary Spark configuration property. | --properties-file FILE Path to a file from which to load extra properties. If not diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 465c214362b25..80dc9bf8779df 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -22,8 +22,6 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, Unparsed} -import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} - import org.apache.spark.internal.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala index 113037d1ab5be..4f3e42bb3c94e 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala @@ -17,12 +17,6 @@ package org.apache.spark.internal.config -import java.util.{Map => JMap} - -import scala.util.matching.Regex - -import org.apache.spark.SparkConf - /** * An entry contains all meta information for a configuration. * @@ -34,7 +28,6 @@ import org.apache.spark.SparkConf * value declared as a string. * * @param key the key for the configuration - * @param defaultValue the default value for the configuration * @param valueConverter how to convert a string to the value. It should throw an exception if the * string does not have the required format. * @param stringConverter how to convert a value to a string that the user can use it as a valid @@ -76,7 +69,7 @@ private class ConfigEntryWithDefault[T] ( stringConverter: T => String, doc: String, isPublic: Boolean) - extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic) { + extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic) { override def defaultValue: Option[T] = Some(_defaultValue) @@ -95,7 +88,7 @@ private class ConfigEntryWithDefaultString[T] ( stringConverter: T => String, doc: String, isPublic: Boolean) - extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic) { + extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic) { override def defaultValue: Option[T] = Some(valueConverter(_defaultValue)) @@ -118,8 +111,8 @@ private[spark] class OptionalConfigEntry[T]( val rawStringConverter: T => String, doc: String, isPublic: Boolean) - extends ConfigEntry[Option[T]](key, s => Some(rawValueConverter(s)), - v => v.map(rawStringConverter).orNull, doc, isPublic) { + extends ConfigEntry[Option[T]](key, s => Some(rawValueConverter(s)), + v => v.map(rawStringConverter).orNull, doc, isPublic) { override def defaultValueString: String = "" @@ -137,7 +130,7 @@ private class FallbackConfigEntry[T] ( doc: String, isPublic: Boolean, private[config] val fallback: ConfigEntry[T]) - extends ConfigEntry[T](key, fallback.valueConverter, fallback.stringConverter, doc, isPublic) { + extends ConfigEntry[T](key, fallback.valueConverter, fallback.stringConverter, doc, isPublic) { override def defaultValueString: String = s"" diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala index bb1a3bb5fc56f..c62de9bfd8fc3 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala @@ -18,7 +18,6 @@ package org.apache.spark.internal.config import java.util.{Map => JMap} -import java.util.regex.Pattern import scala.collection.mutable.HashMap import scala.util.matching.Regex diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index aba429bcdca60..536f493b41722 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -243,4 +243,20 @@ package object config { "and event logs.") .stringConf .createWithDefault("(?i)secret|password") + + private[spark] val NETWORK_AUTH_ENABLED = + ConfigBuilder("spark.authenticate") + .booleanConf + .createWithDefault(false) + + private[spark] val SASL_ENCRYPTION_ENABLED = + ConfigBuilder("spark.authenticate.enableSaslEncryption") + .booleanConf + .createWithDefault(false) + + private[spark] val NETWORK_ENCRYPTION_ENABLED = + ConfigBuilder("spark.network.crypto.enabled") + .booleanConf + .createWithDefault(false) + } diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index b2d9b8d2a012f..2f33f2e4ff8d1 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -99,7 +99,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) } private def getFilename(taskContext: TaskAttemptContext, ext: String): String = { - // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet + // The file name looks like part-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003-c000.parquet // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, // the file name is fine and won't overflow. val split = taskContext.getTaskAttemptID.getTaskID.getId diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 3d4ea3cccc934..b75e91b660969 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -27,7 +27,7 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.{RpcResponseCallback, TransportClientBootstrap, TransportClientFactory} -import org.apache.spark.network.sasl.{SaslClientBootstrap, SaslServerBootstrap} +import org.apache.spark.network.crypto.{AuthClientBootstrap, AuthServerBootstrap} import org.apache.spark.network.server._ import org.apache.spark.network.shuffle.{BlockFetchingListener, OneForOneBlockFetcher, RetryingBlockFetcher} import org.apache.spark.network.shuffle.protocol.UploadBlock @@ -63,9 +63,8 @@ private[spark] class NettyBlockTransferService( var serverBootstrap: Option[TransportServerBootstrap] = None var clientBootstrap: Option[TransportClientBootstrap] = None if (authEnabled) { - serverBootstrap = Some(new SaslServerBootstrap(transportConf, securityManager)) - clientBootstrap = Some(new SaslClientBootstrap(transportConf, conf.getAppId, securityManager, - securityManager.isSaslEncryptionEnabled())) + serverBootstrap = Some(new AuthServerBootstrap(transportConf, securityManager)) + clientBootstrap = Some(new AuthClientBootstrap(transportConf, conf.getAppId, securityManager)) } transportContext = new TransportContext(transportConf, rpcHandler) clientFactory = transportContext.createClientFactory(clientBootstrap.toSeq.asJava) diff --git a/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala b/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala index 9ba476d2ba26a..ff2f58d81142d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala +++ b/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala @@ -41,9 +41,10 @@ private[spark] object InputFileBlockHolder { * 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 inputBlock: ThreadLocal[FileBlock] = new ThreadLocal[FileBlock] { - override protected def initialValue(): FileBlock = new FileBlock - } + private[this] val inputBlock: InheritableThreadLocal[FileBlock] = + new InheritableThreadLocal[FileBlock] { + override protected def initialValue(): FileBlock = new FileBlock + } /** * Returns the holding file name or empty string if it is unknown. 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 ec12b9963e7c2..d7bfdbad8442f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -109,7 +109,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * functions. This method is here for backward compatibility. It does not provide combiner * classtag information to the shuffle. * - * @see [[combineByKeyWithClassTag]] + * @see `combineByKeyWithClassTag` */ def combineByKey[C]( createCombiner: V => C, @@ -127,7 +127,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * This method is here for backward compatibility. It does not provide combiner * classtag information to the shuffle. * - * @see [[combineByKeyWithClassTag]] + * @see `combineByKeyWithClassTag` */ def combineByKey[C]( createCombiner: V => C, @@ -608,7 +608,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * existing partitioner/parallelism level. This method is here for backward compatibility. It * does not provide combiner classtag information to the shuffle. * - * @see [[combineByKeyWithClassTag]] + * @see `combineByKeyWithClassTag` */ def combineByKey[C]( createCombiner: V => C, 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 374abccf6ad55..0359508c00395 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1610,14 +1610,15 @@ abstract class RDD[T: ClassTag]( /** * Return whether this RDD is checkpointed and materialized, either reliably or locally. */ - def isCheckpointed: Boolean = checkpointData.exists(_.isCheckpointed) + def isCheckpointed: Boolean = isCheckpointedAndMaterialized /** * Return whether this RDD is checkpointed and materialized, either reliably or locally. * This is introduced as an alias for `isCheckpointed` to clarify the semantics of the * return value. Exposed for testing. */ - private[spark] def isCheckpointedAndMaterialized: Boolean = isCheckpointed + private[spark] def isCheckpointedAndMaterialized: Boolean = + checkpointData.exists(_.isCheckpointed) /** * Return whether this RDD is marked for local checkpointing. @@ -1841,7 +1842,7 @@ abstract class RDD[T: ClassTag]( * Defines implicit functions that provide extra functionalities on RDDs of specific types. * * For example, [[RDD.rddToPairRDDFunctions]] converts an RDD into a [[PairRDDFunctions]] for - * key-value-pair RDDs, and enabling extra functionalities such as [[PairRDDFunctions.reduceByKey]]. + * key-value-pair RDDs, and enabling extra functionalities such as `PairRDDFunctions.reduceByKey`. */ object RDD { diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala index b9db60a7797d8..fdbccc9e74c37 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala @@ -25,10 +25,11 @@ import org.apache.spark.SparkException * The `rpcAddress` may be null, in which case the endpoint is registered via a client-only * connection and can only be reached via the client that sent the endpoint reference. * - * @param rpcAddress The socket address of the endpoint. + * @param rpcAddress The socket address of the endpoint. It's `null` when this address pointing to + * an endpoint in a client `NettyRpcEnv`. * @param name Name of the endpoint. */ -private[spark] case class RpcEndpointAddress(val rpcAddress: RpcAddress, val name: String) { +private[spark] case class RpcEndpointAddress(rpcAddress: RpcAddress, name: String) { require(name != null, "RpcEndpoint name must be provided.") diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala index efd26486abaec..2c9a976e76939 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala @@ -19,11 +19,10 @@ package org.apache.spark.rpc import java.util.concurrent.TimeoutException -import scala.concurrent.{Await, Future} +import scala.concurrent.Future import scala.concurrent.duration._ -import scala.util.control.NonFatal -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.SparkConf import org.apache.spark.util.{ThreadUtils, Utils} /** diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index e56943da1303a..ff5e39a8dcbc8 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -33,12 +33,12 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.Logging import org.apache.spark.network.TransportContext import org.apache.spark.network.client._ +import org.apache.spark.network.crypto.{AuthClientBootstrap, AuthServerBootstrap} import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.sasl.{SaslClientBootstrap, SaslServerBootstrap} import org.apache.spark.network.server._ import org.apache.spark.rpc._ -import org.apache.spark.serializer.{JavaSerializer, JavaSerializerInstance} -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.serializer.{JavaSerializer, JavaSerializerInstance, SerializationStream} +import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, ThreadUtils, Utils} private[netty] class NettyRpcEnv( val conf: SparkConf, @@ -60,8 +60,8 @@ private[netty] class NettyRpcEnv( private def createClientBootstraps(): java.util.List[TransportClientBootstrap] = { if (securityManager.isAuthenticationEnabled()) { - java.util.Arrays.asList(new SaslClientBootstrap(transportConf, "", securityManager, - securityManager.isSaslEncryptionEnabled())) + java.util.Arrays.asList(new AuthClientBootstrap(transportConf, + securityManager.getSaslUser(), securityManager)) } else { java.util.Collections.emptyList[TransportClientBootstrap] } @@ -111,7 +111,7 @@ private[netty] class NettyRpcEnv( def startServer(bindAddress: String, port: Int): Unit = { val bootstraps: java.util.List[TransportServerBootstrap] = if (securityManager.isAuthenticationEnabled()) { - java.util.Arrays.asList(new SaslServerBootstrap(transportConf, securityManager)) + java.util.Arrays.asList(new AuthServerBootstrap(transportConf, securityManager)) } else { java.util.Collections.emptyList() } @@ -189,7 +189,7 @@ private[netty] class NettyRpcEnv( } } else { // Message to a remote RPC endpoint. - postToOutbox(message.receiver, OneWayOutboxMessage(serialize(message))) + postToOutbox(message.receiver, OneWayOutboxMessage(message.serialize(this))) } } @@ -224,7 +224,7 @@ private[netty] class NettyRpcEnv( }(ThreadUtils.sameThread) dispatcher.postLocalMessage(message, p) } else { - val rpcMessage = RpcOutboxMessage(serialize(message), + val rpcMessage = RpcOutboxMessage(message.serialize(this), onFailure, (client, response) => onSuccess(deserialize[Any](client, response))) postToOutbox(message.receiver, rpcMessage) @@ -253,6 +253,13 @@ private[netty] class NettyRpcEnv( javaSerializerInstance.serialize(content) } + /** + * Returns [[SerializationStream]] that forwards the serialized bytes to `out`. + */ + private[netty] def serializeStream(out: OutputStream): SerializationStream = { + javaSerializerInstance.serializeStream(out) + } + private[netty] def deserialize[T: ClassTag](client: TransportClient, bytes: ByteBuffer): T = { NettyRpcEnv.currentClient.withValue(client) { deserialize { () => @@ -480,16 +487,13 @@ private[rpc] class NettyRpcEnvFactory extends RpcEnvFactory with Logging { */ private[netty] class NettyRpcEndpointRef( @transient private val conf: SparkConf, - endpointAddress: RpcEndpointAddress, - @transient @volatile private var nettyEnv: NettyRpcEnv) - extends RpcEndpointRef(conf) with Serializable with Logging { + private val endpointAddress: RpcEndpointAddress, + @transient @volatile private var nettyEnv: NettyRpcEnv) extends RpcEndpointRef(conf) { @transient @volatile var client: TransportClient = _ - private val _address = if (endpointAddress.rpcAddress != null) endpointAddress else null - private val _name = endpointAddress.name - - override def address: RpcAddress = if (_address != null) _address.rpcAddress else null + override def address: RpcAddress = + if (endpointAddress.rpcAddress != null) endpointAddress.rpcAddress else null private def readObject(in: ObjectInputStream): Unit = { in.defaultReadObject() @@ -501,34 +505,103 @@ private[netty] class NettyRpcEndpointRef( out.defaultWriteObject() } - override def name: String = _name + override def name: String = endpointAddress.name override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = { - nettyEnv.ask(RequestMessage(nettyEnv.address, this, message), timeout) + nettyEnv.ask(new RequestMessage(nettyEnv.address, this, message), timeout) } override def send(message: Any): Unit = { require(message != null, "Message is null") - nettyEnv.send(RequestMessage(nettyEnv.address, this, message)) + nettyEnv.send(new RequestMessage(nettyEnv.address, this, message)) } - override def toString: String = s"NettyRpcEndpointRef(${_address})" - - def toURI: URI = new URI(_address.toString) + override def toString: String = s"NettyRpcEndpointRef(${endpointAddress})" final override def equals(that: Any): Boolean = that match { - case other: NettyRpcEndpointRef => _address == other._address + case other: NettyRpcEndpointRef => endpointAddress == other.endpointAddress case _ => false } - final override def hashCode(): Int = if (_address == null) 0 else _address.hashCode() + final override def hashCode(): Int = + if (endpointAddress == null) 0 else endpointAddress.hashCode() } /** * The message that is sent from the sender to the receiver. + * + * @param senderAddress the sender address. It's `null` if this message is from a client + * `NettyRpcEnv`. + * @param receiver the receiver of this message. + * @param content the message content. */ -private[netty] case class RequestMessage( - senderAddress: RpcAddress, receiver: NettyRpcEndpointRef, content: Any) +private[netty] class RequestMessage( + val senderAddress: RpcAddress, + val receiver: NettyRpcEndpointRef, + val content: Any) { + + /** Manually serialize [[RequestMessage]] to minimize the size. */ + def serialize(nettyEnv: NettyRpcEnv): ByteBuffer = { + val bos = new ByteBufferOutputStream() + val out = new DataOutputStream(bos) + try { + writeRpcAddress(out, senderAddress) + writeRpcAddress(out, receiver.address) + out.writeUTF(receiver.name) + val s = nettyEnv.serializeStream(out) + try { + s.writeObject(content) + } finally { + s.close() + } + } finally { + out.close() + } + bos.toByteBuffer + } + + private def writeRpcAddress(out: DataOutputStream, rpcAddress: RpcAddress): Unit = { + if (rpcAddress == null) { + out.writeBoolean(false) + } else { + out.writeBoolean(true) + out.writeUTF(rpcAddress.host) + out.writeInt(rpcAddress.port) + } + } + + override def toString: String = s"RequestMessage($senderAddress, $receiver, $content)" +} + +private[netty] object RequestMessage { + + private def readRpcAddress(in: DataInputStream): RpcAddress = { + val hasRpcAddress = in.readBoolean() + if (hasRpcAddress) { + RpcAddress(in.readUTF(), in.readInt()) + } else { + null + } + } + + def apply(nettyEnv: NettyRpcEnv, client: TransportClient, bytes: ByteBuffer): RequestMessage = { + val bis = new ByteBufferInputStream(bytes) + val in = new DataInputStream(bis) + try { + val senderAddress = readRpcAddress(in) + val endpointAddress = RpcEndpointAddress(readRpcAddress(in), in.readUTF()) + val ref = new NettyRpcEndpointRef(nettyEnv.conf, endpointAddress, nettyEnv) + ref.client = client + new RequestMessage( + senderAddress, + ref, + // The remaining bytes in `bytes` are the message content. + nettyEnv.deserialize(client, bytes)) + } finally { + in.close() + } + } +} /** * A response that indicates some failure happens in the receiver side. @@ -574,10 +647,10 @@ private[netty] class NettyRpcHandler( val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] assert(addr != null) val clientAddr = RpcAddress(addr.getHostString, addr.getPort) - val requestMessage = nettyEnv.deserialize[RequestMessage](client, message) + val requestMessage = RequestMessage(nettyEnv, client, message) if (requestMessage.senderAddress == null) { // Create a new message with the socket address of the client as the sender. - RequestMessage(clientAddr, requestMessage.receiver, requestMessage.content) + new RequestMessage(clientAddr, requestMessage.receiver, requestMessage.content) } else { // The remote RpcEnv listens to some port, we should also fire a RemoteProcessConnected for // the listening address diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index bf7a62ea33875..77d5c97a32711 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicReference import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.util.{Clock, SystemClock, Utils} @@ -48,9 +48,14 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} * one exception is [[nodeBlacklist()]], which can be called without holding a lock. */ private[scheduler] class BlacklistTracker ( + private val listenerBus: LiveListenerBus, conf: SparkConf, clock: Clock = new SystemClock()) extends Logging { + def this(sc: SparkContext) = { + this(sc.listenerBus, sc.conf) + } + BlacklistTracker.validateBlacklistConfs(conf) private val MAX_FAILURES_PER_EXEC = conf.get(config.MAX_FAILURES_PER_EXEC) private val MAX_FAILED_EXEC_PER_NODE = conf.get(config.MAX_FAILED_EXEC_PER_NODE) @@ -103,6 +108,7 @@ private[scheduler] class BlacklistTracker ( execsToUnblacklist.foreach { exec => val status = executorIdToBlacklistStatus.remove(exec).get val failedExecsOnNode = nodeToBlacklistedExecs(status.node) + listenerBus.post(SparkListenerExecutorUnblacklisted(now, exec)) failedExecsOnNode.remove(exec) if (failedExecsOnNode.isEmpty) { nodeToBlacklistedExecs.remove(status.node) @@ -114,7 +120,10 @@ private[scheduler] class BlacklistTracker ( // Un-blacklist any nodes that have been blacklisted longer than the blacklist timeout. logInfo(s"Removing nodes $nodesToUnblacklist from blacklist because the blacklist " + s"has timed out") - nodeIdToBlacklistExpiryTime --= nodesToUnblacklist + nodesToUnblacklist.foreach { node => + nodeIdToBlacklistExpiryTime.remove(node) + listenerBus.post(SparkListenerNodeUnblacklisted(now, node)) + } _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } updateNextExpiryTime() @@ -161,6 +170,8 @@ private[scheduler] class BlacklistTracker ( s" task failures in successful task sets") val node = failuresInTaskSet.node executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTimeForNewBlacklists)) + listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, newTotal)) + executorIdToFailureList.remove(exec) updateNextExpiryTime() // In addition to blacklisting the executor, we also update the data for failures on the @@ -174,6 +185,7 @@ private[scheduler] class BlacklistTracker ( logInfo(s"Blacklisting node $node because it has ${blacklistedExecsOnNode.size} " + s"executors blacklisted: ${blacklistedExecsOnNode}") nodeIdToBlacklistExpiryTime.put(node, expiryTimeForNewBlacklists) + listenerBus.post(SparkListenerNodeBlacklisted(now, node, blacklistedExecsOnNode.size)) _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index f39565edd2358..af9bdefc967ef 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -193,6 +193,22 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) } + override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = { + logEvent(event, flushLogger = true) + } + + override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = { + logEvent(event, flushLogger = true) + } + + override def onNodeBlacklisted(event: SparkListenerNodeBlacklisted): Unit = { + logEvent(event, flushLogger = true) + } + + override def onNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Unit = { + logEvent(event, flushLogger = true) + } + // No-op because logging every update would be overkill override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {} diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index 7bed6851d0cde..08d220b40b6f3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -22,6 +22,7 @@ import scala.collection.mutable import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} +import org.apache.spark.util.{RpcUtils, ThreadUtils} private sealed trait OutputCommitCoordinationMessage extends Serializable @@ -88,7 +89,8 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) val msg = AskPermissionToCommitOutput(stage, partition, attemptNumber) coordinatorRef match { case Some(endpointRef) => - endpointRef.askWithRetry[Boolean](msg) + ThreadUtils.awaitResult(endpointRef.ask[Boolean](msg), + RpcUtils.askRpcTimeout(conf).duration) case None => logError( "canCommit called after coordinator was stopped (is SparkEnv shutdown in progress)?") @@ -165,9 +167,18 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) authorizedCommitters(partition) = attemptNumber true case existingCommitter => - logDebug(s"Denying attemptNumber=$attemptNumber to commit for stage=$stage, " + - s"partition=$partition; existingCommitter = $existingCommitter") - false + // Coordinator should be idempotent when receiving AskPermissionToCommit. + if (existingCommitter == attemptNumber) { + logWarning(s"Authorizing duplicate request to commit for " + + s"attemptNumber=$attemptNumber to commit for stage=$stage," + + s" partition=$partition; existingCommitter = $existingCommitter." + + s" This can indicate dropped network traffic.") + true + } else { + logDebug(s"Denying attemptNumber=$attemptNumber to commit for stage=$stage, " + + s"partition=$partition; existingCommitter = $existingCommitter") + false + } } case None => logDebug(s"Stage $stage has completed, so not allowing attempt number $attemptNumber of" + diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 6abdf0fd53968..e36c759a42556 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -24,7 +24,6 @@ import java.util.Properties import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 994b81e0624ec..7a25c47e2cab3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -25,7 +25,6 @@ import scala.language.existentials import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.ShuffleWriter diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 7618dfeeedf8d..1b12af75864ea 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -105,6 +105,28 @@ case class SparkListenerExecutorAdded(time: Long, executorId: String, executorIn case class SparkListenerExecutorRemoved(time: Long, executorId: String, reason: String) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerExecutorBlacklisted( + time: Long, + executorId: String, + taskFailures: Int) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerExecutorUnblacklisted(time: Long, executorId: String) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerNodeBlacklisted( + time: Long, + hostId: String, + executorFailures: Int) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerNodeUnblacklisted(time: Long, hostId: String) + extends SparkListenerEvent + @DeveloperApi case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends SparkListenerEvent @@ -238,6 +260,26 @@ private[spark] trait SparkListenerInterface { */ def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit + /** + * Called when the driver blacklists an executor for a Spark application. + */ + def onExecutorBlacklisted(executorBlacklisted: SparkListenerExecutorBlacklisted): Unit + + /** + * Called when the driver re-enables a previously blacklisted executor. + */ + def onExecutorUnblacklisted(executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit + + /** + * Called when the driver blacklists a node for a Spark application. + */ + def onNodeBlacklisted(nodeBlacklisted: SparkListenerNodeBlacklisted): Unit + + /** + * Called when the driver re-enables a previously blacklisted node. + */ + def onNodeUnblacklisted(nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit + /** * Called when the driver receives a block update info. */ @@ -293,6 +335,18 @@ abstract class SparkListener extends SparkListenerInterface { override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { } + override def onExecutorBlacklisted( + executorBlacklisted: SparkListenerExecutorBlacklisted): Unit = { } + + override def onExecutorUnblacklisted( + executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit = { } + + override def onNodeBlacklisted( + nodeBlacklisted: SparkListenerNodeBlacklisted): Unit = { } + + override def onNodeUnblacklisted( + nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit = { } + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { } override def onOtherEvent(event: SparkListenerEvent): Unit = { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 471586ac0852a..3ff363321e8c9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -61,6 +61,14 @@ private[spark] trait SparkListenerBus listener.onExecutorAdded(executorAdded) case executorRemoved: SparkListenerExecutorRemoved => listener.onExecutorRemoved(executorRemoved) + case executorBlacklisted: SparkListenerExecutorBlacklisted => + listener.onExecutorBlacklisted(executorBlacklisted) + case executorUnblacklisted: SparkListenerExecutorUnblacklisted => + listener.onExecutorUnblacklisted(executorUnblacklisted) + case nodeBlacklisted: SparkListenerNodeBlacklisted => + listener.onNodeBlacklisted(nodeBlacklisted) + case nodeUnblacklisted: SparkListenerNodeUnblacklisted => + listener.onNodeUnblacklisted(nodeUnblacklisted) case blockUpdated: SparkListenerBlockUpdated => listener.onBlockUpdated(blockUpdated) case logStart: SparkListenerLogStart => // ignore event log metadata diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 2f972b064b477..c628dd38d07bb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashSet -import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD 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 51976f666dfce..7b726d5659e91 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -46,7 +46,6 @@ import org.apache.spark.util._ * @param stageId id of the stage this task belongs to * @param stageAttemptId attempt id of the stage this task belongs to * @param partitionId index of the number in the RDD - * @param metrics a `TaskMetrics` that is created at driver side and sent to executor side. * @param localProperties copy of thread-local properties set by the user on the driver side. * @param serializedTaskMetrics a `TaskMetrics` that is created and serialized on the driver side * and sent to executor side. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 9a8e313f9ec7b..72ed55af41f4d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -63,14 +63,14 @@ private[spark] class TaskSchedulerImpl private[scheduler]( this( sc, sc.conf.get(config.MAX_TASK_FAILURES), - TaskSchedulerImpl.maybeCreateBlacklistTracker(sc.conf)) + TaskSchedulerImpl.maybeCreateBlacklistTracker(sc)) } def this(sc: SparkContext, maxTaskFailures: Int, isLocal: Boolean) = { this( sc, maxTaskFailures, - TaskSchedulerImpl.maybeCreateBlacklistTracker(sc.conf), + TaskSchedulerImpl.maybeCreateBlacklistTracker(sc), isLocal = isLocal) } @@ -717,9 +717,9 @@ private[spark] object TaskSchedulerImpl { retval.toList } - private def maybeCreateBlacklistTracker(conf: SparkConf): Option[BlacklistTracker] = { - if (BlacklistTracker.isBlacklistEnabled(conf)) { - Some(new BlacklistTracker(conf)) + private def maybeCreateBlacklistTracker(sc: SparkContext): Option[BlacklistTracker] = { + if (BlacklistTracker.isBlacklistEnabled(sc.conf)) { + Some(new BlacklistTracker(sc)) } else { None } diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index afe6cd86059f0..008b0387899f6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -23,7 +23,6 @@ import javax.annotation.concurrent.NotThreadSafe import scala.reflect.ClassTag -import org.apache.spark.SparkEnv import org.apache.spark.annotation.{DeveloperApi, Private} import org.apache.spark.util.NextIterator diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 686305e9335dc..748f0a30ada96 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -23,7 +23,6 @@ import java.nio.ByteBuffer import scala.reflect.ClassTag import org.apache.spark.SparkConf -import org.apache.spark.internal.config._ import org.apache.spark.io.CompressionCodec import org.apache.spark.security.CryptoStreamUtils import org.apache.spark.storage._ diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index acb7c23079681..1818935392eb3 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -142,8 +142,10 @@ private[v1] object AllStagesResource { index = uiData.taskInfo.index, attempt = uiData.taskInfo.attemptNumber, launchTime = new Date(uiData.taskInfo.launchTime), + duration = uiData.taskDuration, executorId = uiData.taskInfo.executorId, host = uiData.taskInfo.host, + status = uiData.taskInfo.status, taskLocality = uiData.taskInfo.taskLocality.toString(), speculative = uiData.taskInfo.speculative, accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo }, diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala index 76779290d45e6..a0239266d8756 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -30,6 +30,8 @@ private[v1] class ApplicationListResource(uiRoot: UIRoot) { @QueryParam("status") status: JList[ApplicationStatus], @DefaultValue("2010-01-01") @QueryParam("minDate") minDate: SimpleDateParam, @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam, + @DefaultValue("2010-01-01") @QueryParam("minEndDate") minEndDate: SimpleDateParam, + @DefaultValue("3000-01-01") @QueryParam("maxEndDate") maxEndDate: SimpleDateParam, @QueryParam("limit") limit: Integer) : Iterator[ApplicationInfo] = { @@ -43,11 +45,27 @@ private[v1] class ApplicationListResource(uiRoot: UIRoot) { // keep the app if *any* attempts fall in the right time window ((!anyRunning && includeCompleted) || (anyRunning && includeRunning)) && app.attempts.exists { attempt => - val start = attempt.startTime.getTime - start >= minDate.timestamp && start <= maxDate.timestamp + isAttemptInRange(attempt, minDate, maxDate, minEndDate, maxEndDate, anyRunning) } }.take(numApps) } + + private def isAttemptInRange( + attempt: ApplicationAttemptInfo, + minStartDate: SimpleDateParam, + maxStartDate: SimpleDateParam, + minEndDate: SimpleDateParam, + maxEndDate: SimpleDateParam, + anyRunning: Boolean): Boolean = { + val startTimeOk = attempt.startTime.getTime >= minStartDate.timestamp && + attempt.startTime.getTime <= maxStartDate.timestamp + // If the maxEndDate is in the past, exclude all running apps. + val endTimeOkForRunning = anyRunning && (maxEndDate.timestamp > System.currentTimeMillis()) + val endTimeOkForCompleted = !anyRunning && (attempt.endTime.getTime >= minEndDate.timestamp && + attempt.endTime.getTime <= maxEndDate.timestamp) + val endTimeOk = endTimeOkForRunning || endTimeOkForCompleted + startTimeOk && endTimeOk + } } private[spark] object ApplicationsListResource { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala index 6ca59c2f3caeb..ab53881594180 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.status.api.v1 -import javax.ws.rs.{GET, PathParam, Produces} +import javax.ws.rs.{GET, Produces} import javax.ws.rs.core.MediaType import org.apache.spark.ui.SparkUI diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 44a929b310384..c509398db1ecf 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -73,6 +73,7 @@ class ExecutorSummary private[spark]( val totalInputBytes: Long, val totalShuffleRead: Long, val totalShuffleWrite: Long, + val isBlacklisted: Boolean, val maxMemory: Long, val executorLogs: Map[String, String]) @@ -157,8 +158,10 @@ class TaskData private[spark]( val index: Int, val attempt: Int, val launchTime: Date, + val duration: Option[Long] = None, val executorId: String, val host: String, + val status: String, val taskLocality: String, val speculative: Boolean, val accumulatorUpdates: Seq[AccumulableInfo], diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 04521c9159eac..c40186756f2d5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -125,8 +125,7 @@ private[spark] class BlockManager( // standard BlockTransferService to directly connect to other Executors. private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores) - new ExternalShuffleClient(transConf, securityManager, securityManager.isAuthenticationEnabled(), - securityManager.isSaslEncryptionEnabled()) + new ExternalShuffleClient(transConf, securityManager, securityManager.isAuthenticationEnabled()) } else { blockTransferService } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index 3cb12fca7dccb..eb3ff926372a2 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -128,16 +128,19 @@ private[spark] class DiskBlockObjectWriter( */ private def closeResources(): Unit = { if (initialized) { - mcs.manualClose() - channel = null - mcs = null - bs = null - fos = null - ts = null - objOut = null - initialized = false - streamOpen = false - hasBeenClosed = true + Utils.tryWithSafeFinally { + mcs.manualClose() + } { + channel = null + mcs = null + bs = null + fos = null + ts = null + objOut = null + initialized = false + streamOpen = false + hasBeenClosed = true + } } } @@ -199,26 +202,29 @@ private[spark] class DiskBlockObjectWriter( def revertPartialWritesAndClose(): File = { // Discard current writes. We do this by flushing the outstanding writes and then // truncating the file to its initial position. - try { + Utils.tryWithSafeFinally { if (initialized) { writeMetrics.decBytesWritten(reportedPosition - committedPosition) writeMetrics.decRecordsWritten(numRecordsWritten) streamOpen = false closeResources() } - - val truncateStream = new FileOutputStream(file, true) + } { + var truncateStream: FileOutputStream = null try { + truncateStream = new FileOutputStream(file, true) truncateStream.getChannel.truncate(committedPosition) - file + } catch { + case e: Exception => + logError("Uncaught exception while reverting partial writes to file " + file, e) } finally { - truncateStream.close() + if (truncateStream != null) { + truncateStream.close() + truncateStream = null + } } - } catch { - case e: Exception => - logError("Uncaught exception while reverting partial writes to file " + file, e) - file } + file } /** diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index fad0404bebc36..4c6998d7a8e20 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils * ExternalBlockStore, whether to keep the data in memory in a serialized format, and whether * to replicate the RDD partitions on multiple nodes. * - * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants + * The [[org.apache.spark.storage.StorageLevel]] singleton object contains some static constants * for commonly useful storage levels. To create your own storage level object, use the * factory method of the singleton object (`StorageLevel(...)`). */ diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 35c3c8d00f99b..f713619cd7ec4 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -45,6 +45,9 @@ import org.apache.spark.util.Utils */ private[spark] object JettyUtils extends Logging { + val SPARK_CONNECTOR_NAME = "Spark" + val REDIRECT_CONNECTOR_NAME = "HttpsRedirect" + // Base type for a function that returns something based on an HTTP request. Allows for // implicit conversion from many types of functions to jetty Handlers. type Responder[T] = HttpServletRequest => T @@ -274,17 +277,18 @@ private[spark] object JettyUtils extends Logging { conf: SparkConf, serverName: String = ""): ServerInfo = { - val collection = new ContextHandlerCollection addFilters(handlers, conf) val gzipHandlers = handlers.map { h => + h.setVirtualHosts(Array("@" + SPARK_CONNECTOR_NAME)) + val gzipHandler = new GzipHandler gzipHandler.setHandler(h) gzipHandler } // Bind to the given port, or throw a java.net.BindException if the port is occupied - def connect(currentPort: Int): (Server, Int) = { + def connect(currentPort: Int): ((Server, Option[Int]), Int) = { val pool = new QueuedThreadPool if (serverName.nonEmpty) { pool.setName(serverName) @@ -292,7 +296,9 @@ private[spark] object JettyUtils extends Logging { pool.setDaemon(true) val server = new Server(pool) - val connectors = new ArrayBuffer[ServerConnector] + val connectors = new ArrayBuffer[ServerConnector]() + val collection = new ContextHandlerCollection + // Create a connector on port currentPort to listen for HTTP requests val httpConnector = new ServerConnector( server, @@ -306,26 +312,33 @@ private[spark] object JettyUtils extends Logging { httpConnector.setPort(currentPort) connectors += httpConnector - sslOptions.createJettySslContextFactory().foreach { factory => - // If the new port wraps around, do not try a privileged port. - val securePort = - if (currentPort != 0) { - (currentPort + 400 - 1024) % (65536 - 1024) + 1024 - } else { - 0 - } - val scheme = "https" - // Create a connector on port securePort to listen for HTTPS requests - val connector = new ServerConnector(server, factory) - connector.setPort(securePort) - - connectors += connector - - // redirect the HTTP requests to HTTPS port - collection.addHandler(createRedirectHttpsHandler(securePort, scheme)) + val httpsConnector = sslOptions.createJettySslContextFactory() match { + case Some(factory) => + // If the new port wraps around, do not try a privileged port. + val securePort = + if (currentPort != 0) { + (currentPort + 400 - 1024) % (65536 - 1024) + 1024 + } else { + 0 + } + val scheme = "https" + // Create a connector on port securePort to listen for HTTPS requests + val connector = new ServerConnector(server, factory) + connector.setPort(securePort) + connector.setName(SPARK_CONNECTOR_NAME) + connectors += connector + + // redirect the HTTP requests to HTTPS port + httpConnector.setName(REDIRECT_CONNECTOR_NAME) + collection.addHandler(createRedirectHttpsHandler(securePort, scheme)) + Some(connector) + + case None => + // No SSL, so the HTTP connector becomes the official one where all contexts bind. + httpConnector.setName(SPARK_CONNECTOR_NAME) + None } - gzipHandlers.foreach(collection.addHandler) // As each acceptor and each selector will use one thread, the number of threads should at // least be the number of acceptors and selectors plus 1. (See SPARK-13776) var minThreads = 1 @@ -337,17 +350,20 @@ private[spark] object JettyUtils extends Logging { // The number of selectors always equals to the number of acceptors minThreads += connector.getAcceptors * 2 } - server.setConnectors(connectors.toArray) pool.setMaxThreads(math.max(pool.getMaxThreads, minThreads)) val errorHandler = new ErrorHandler() errorHandler.setShowStacks(true) errorHandler.setServer(server) server.addBean(errorHandler) + + gzipHandlers.foreach(collection.addHandler) server.setHandler(collection) + + server.setConnectors(connectors.toArray) try { server.start() - (server, httpConnector.getLocalPort) + ((server, httpsConnector.map(_.getLocalPort())), httpConnector.getLocalPort) } catch { case e: Exception => server.stop() @@ -356,13 +372,16 @@ private[spark] object JettyUtils extends Logging { } } - val (server, boundPort) = Utils.startServiceOnPort[Server](port, connect, conf, serverName) - ServerInfo(server, boundPort, collection) + val ((server, securePort), boundPort) = Utils.startServiceOnPort(port, connect, conf, + serverName) + ServerInfo(server, boundPort, securePort, + server.getHandler().asInstanceOf[ContextHandlerCollection]) } private def createRedirectHttpsHandler(securePort: Int, scheme: String): ContextHandler = { val redirectHandler: ContextHandler = new ContextHandler redirectHandler.setContextPath("/") + redirectHandler.setVirtualHosts(Array("@" + REDIRECT_CONNECTOR_NAME)) redirectHandler.setHandler(new AbstractHandler { override def handle( target: String, @@ -442,7 +461,23 @@ private[spark] object JettyUtils extends Logging { private[spark] case class ServerInfo( server: Server, boundPort: Int, - rootHandler: ContextHandlerCollection) { + securePort: Option[Int], + private val rootHandler: ContextHandlerCollection) { + + def addHandler(handler: ContextHandler): Unit = { + handler.setVirtualHosts(Array("@" + JettyUtils.SPARK_CONNECTOR_NAME)) + rootHandler.addHandler(handler) + if (!handler.isStarted()) { + handler.start() + } + } + + def removeHandler(handler: ContextHandler): Unit = { + rootHandler.removeHandler(handler) + if (handler.isStarted) { + handler.stop() + } + } def stop(): Unit = { server.stop() diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 3cc5353f475f4..766cc65084f07 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -91,6 +91,9 @@ private[spark] object ToolTips { val TASK_TIME = "Shaded red when garbage collection (GC) time is over 10% of task time" + val BLACKLISTED = + "Shows if this executor has been blacklisted by the scheduler due to task failures." + val APPLICATION_EXECUTOR_LIMIT = """Maximum number of executors that this application will use. This limit is finite only when dynamic allocation is enabled. The number of granted executors may exceed the limit 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 b8604c52e6b0b..a9480cc220c8d 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -91,23 +91,13 @@ private[spark] abstract class WebUI( /** Attach a handler to this UI. */ def attachHandler(handler: ServletContextHandler) { handlers += handler - serverInfo.foreach { info => - info.rootHandler.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } - } + serverInfo.foreach(_.addHandler(handler)) } /** Detach a handler from this UI. */ def detachHandler(handler: ServletContextHandler) { handlers -= handler - serverInfo.foreach { info => - info.rootHandler.removeHandler(handler) - if (handler.isStarted) { - handler.stop() - } - } + serverInfo.foreach(_.removeHandler(handler)) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 7953d77fd7ece..2d1691e55c428 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -39,6 +39,7 @@ private[ui] case class ExecutorSummaryInfo( totalInputBytes: Long, totalShuffleRead: Long, totalShuffleWrite: Long, + isBlacklisted: Int, maxMemory: Long, executorLogs: Map[String, String]) @@ -101,6 +102,7 @@ private[spark] object ExecutorsPage { taskSummary.inputBytes, taskSummary.shuffleRead, taskSummary.shuffleWrite, + taskSummary.isBlacklisted, maxMem, taskSummary.executorLogs ) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 678571fd4f5ac..8ae712f8ed323 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -53,7 +53,8 @@ private[ui] case class ExecutorTaskSummary( var shuffleRead: Long = 0L, var shuffleWrite: Long = 0L, var executorLogs: Map[String, String] = Map.empty, - var isAlive: Boolean = true + var isAlive: Boolean = true, + var isBlacklisted: Boolean = false ) /** @@ -73,7 +74,8 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar def deadStorageStatusList: Seq[StorageStatus] = storageStatusListener.deadStorageStatusList - override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = synchronized { + override def onExecutorAdded( + executorAdded: SparkListenerExecutorAdded): Unit = synchronized { val eid = executorAdded.executorId val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) taskSummary.executorLogs = executorAdded.executorInfo.logUrlMap @@ -100,7 +102,8 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar executorToTaskSummary.get(executorRemoved.executorId).foreach(e => e.isAlive = false) } - override def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit = { + override def onApplicationStart( + applicationStart: SparkListenerApplicationStart): Unit = { applicationStart.driverLogs.foreach { logs => val storageStatus = activeStorageStatusList.find { s => s.blockManagerId.executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER || @@ -114,13 +117,15 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar } } - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { + override def onTaskStart( + taskStart: SparkListenerTaskStart): Unit = synchronized { val eid = taskStart.taskInfo.executorId val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) taskSummary.tasksActive += 1 } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + override def onTaskEnd( + taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo if (info != null) { val eid = info.executorId @@ -157,4 +162,46 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar } } + private def updateExecutorBlacklist( + eid: String, + isBlacklisted: Boolean): Unit = { + val execTaskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + execTaskSummary.isBlacklisted = isBlacklisted + } + + override def onExecutorBlacklisted( + executorBlacklisted: SparkListenerExecutorBlacklisted) + : Unit = synchronized { + updateExecutorBlacklist(executorBlacklisted.executorId, true) + } + + override def onExecutorUnblacklisted( + executorUnblacklisted: SparkListenerExecutorUnblacklisted) + : Unit = synchronized { + updateExecutorBlacklist(executorUnblacklisted.executorId, false) + } + + override def onNodeBlacklisted( + nodeBlacklisted: SparkListenerNodeBlacklisted) + : Unit = synchronized { + // Implicitly blacklist every executor associated with this node, and show this in the UI. + activeStorageStatusList.foreach { status => + if (status.blockManagerId.host == nodeBlacklisted.hostId) { + updateExecutorBlacklist(status.blockManagerId.executorId, true) + } + } + } + + override def onNodeUnblacklisted( + nodeUnblacklisted: SparkListenerNodeUnblacklisted) + : Unit = synchronized { + // Implicitly unblacklist every executor associated with this node, regardless of how + // they may have been blacklisted initially (either explicitly through executor blacklisting + // or implicitly through node blacklisting). Show this in the UI. + activeStorageStatusList.foreach { status => + if (status.blockManagerId.host == nodeUnblacklisted.hostId) { + updateExecutorBlacklist(status.blockManagerId.executorId, false) + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 9fb3f35fd9685..cd1b02addc789 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -85,6 +85,11 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage Shuffle Spill (Memory) Shuffle Spill (Disk) }} + + + Blacklisted + + {createExecutorTable()} @@ -160,6 +165,7 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage {Utils.bytesToString(v.diskBytesSpilled)} }} + {v.isBlacklisted} } case None => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 83dc5d874589e..e87caff426436 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -142,7 +142,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { /** If stages is too large, remove and garbage collect old stages */ private def trimStagesIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > retainedStages) { - val toRemove = (stages.size - retainedStages) + val toRemove = calculateNumberToRemove(stages.size, retainedStages) stages.take(toRemove).foreach { s => stageIdToData.remove((s.stageId, s.attemptId)) stageIdToInfo.remove(s.stageId) @@ -154,7 +154,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { /** If jobs is too large, remove and garbage collect old jobs */ private def trimJobsIfNecessary(jobs: ListBuffer[JobUIData]) = synchronized { if (jobs.size > retainedJobs) { - val toRemove = (jobs.size - retainedJobs) + val toRemove = calculateNumberToRemove(jobs.size, retainedJobs) jobs.take(toRemove).foreach { job => // Remove the job's UI data, if it exists jobIdToData.remove(job.jobId).foreach { removedJob => @@ -409,7 +409,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // If Tasks is too large, remove and garbage collect old tasks if (stageData.taskData.size > retainedTasks) { - stageData.taskData = stageData.taskData.drop(stageData.taskData.size - retainedTasks) + stageData.taskData = stageData.taskData.drop( + calculateNumberToRemove(stageData.taskData.size, retainedTasks)) } for ( @@ -430,6 +431,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } + /** + * Remove at least (maxRetained / 10) items to reduce friction. + */ + private def calculateNumberToRemove(dataSize: Int, retainedSize: Int): Int = { + math.max(retainedSize / 10, dataSize - retainedSize) + } + /** * Upon receiving new metrics for a task, updates the per-stage and per-executor-per-stage * aggregate metrics by calculating deltas between the currently recorded metrics and the new diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 412ddfa9fad35..ff17775008acc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -887,10 +887,8 @@ private[ui] class TaskDataSource( private def taskRow(taskData: TaskUIData): TaskTableRowData = { val info = taskData.taskInfo val metrics = taskData.metrics - val duration = if (info.status == "RUNNING") info.timeRunning(currentTime) - else metrics.map(_.executorRunTime).getOrElse(1L) - val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) - else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") + val duration = taskData.taskDuration.getOrElse(1L) + val formatDuration = taskData.taskDuration.map(d => UIUtils.formatDuration(d)).getOrElse("") val schedulerDelay = metrics.map(getSchedulerDelay(info, _, currentTime)).getOrElse(0L) val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 9ce8542f02791..201e6197baa9c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -43,6 +43,7 @@ private[spark] object UIData { var shuffleWriteRecords : Long = 0 var memoryBytesSpilled : Long = 0 var diskBytesSpilled : Long = 0 + var isBlacklisted : Int = 0 } class JobUIData( @@ -92,6 +93,7 @@ private[spark] object UIData { var shuffleWriteRecords: Long = _ var memoryBytesSpilled: Long = _ var diskBytesSpilled: Long = _ + var isBlacklisted: Int = _ var schedulingPool: String = "" var description: Option[String] = None @@ -127,6 +129,14 @@ private[spark] object UIData { def updateTaskMetrics(metrics: Option[TaskMetrics]): Unit = { _metrics = TaskUIData.toTaskMetricsUIData(metrics) } + + def taskDuration: Option[Long] = { + if (taskInfo.status == "RUNNING") { + Some(_taskInfo.timeRunning(System.currentTimeMillis)) + } else { + _metrics.map(_.executorRunTime) + } + } } object TaskUIData { diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index 0e330879d50f9..43bfe0aacf35b 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -222,7 +222,12 @@ private[ui] object RDDOperationGraph extends Logging { /** Return the dot representation of a node in an RDDOperationGraph. */ private def makeDotNode(node: RDDOperationNode): String = { - val label = s"${node.name} [${node.id}]\n${node.callsite}" + val isCached = if (node.cached) { + " [Cached]" + } else { + "" + } + val label = s"${node.name} [${node.id}]$isCached\n${node.callsite}" s"""${node.id} [label="${StringEscapeUtils.escapeJava(label)}"]""" } diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 8c67364ef1a05..ea99a7e5b4847 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -19,7 +19,6 @@ package org.apache.spark.util.random import java.util.Random -import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.commons.math3.distribution.PoissonDistribution diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index cba44c848e012..10902ab5a8327 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,32 @@ [ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { @@ -8,8 +36,8 @@ "duration" : 10505, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 } ] }, { @@ -23,8 +51,8 @@ "duration" : 57, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917380950, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -34,8 +62,8 @@ "duration" : 10, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890, + "startTimeEpoch" : 1430917380880, "lastUpdatedEpoch" : 0 } ] }, { @@ -49,8 +77,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426633910242, "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -60,8 +88,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, "lastUpdatedEpoch" : 0 } ] }, { @@ -74,8 +102,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1425081758277, "endTimeEpoch" : 1425081766912, + "startTimeEpoch" : 1425081758277, "lastUpdatedEpoch" : 0 } ] }, { @@ -88,8 +116,8 @@ "duration" : 9011, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981779720, "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, "lastUpdatedEpoch" : 0 } ] }, { @@ -102,8 +130,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index cba44c848e012..10902ab5a8327 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,32 @@ [ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { @@ -8,8 +36,8 @@ "duration" : 10505, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 } ] }, { @@ -23,8 +51,8 @@ "duration" : 57, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917380950, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -34,8 +62,8 @@ "duration" : 10, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890, + "startTimeEpoch" : 1430917380880, "lastUpdatedEpoch" : 0 } ] }, { @@ -49,8 +77,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426633910242, "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -60,8 +88,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, "lastUpdatedEpoch" : 0 } ] }, { @@ -74,8 +102,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1425081758277, "endTimeEpoch" : 1425081766912, + "startTimeEpoch" : 1425081758277, "lastUpdatedEpoch" : 0 } ] }, { @@ -88,8 +116,8 @@ "duration" : 9011, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981779720, "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, "lastUpdatedEpoch" : 0 } ] }, { @@ -102,8 +130,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index e7db6742c25e1..6b9f29e1a230e 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -16,6 +16,7 @@ "totalInputBytes" : 28000288, "totalShuffleRead" : 0, "totalShuffleWrite" : 13180, + "isBlacklisted" : false, "maxMemory" : 278302556, "executorLogs" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json new file mode 100644 index 0000000000000..5914a1c2c4b6d --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -0,0 +1,118 @@ +[ { + "id" : "2", + "hostPort" : "172.22.0.167:51487", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2537, + "totalGCTime" : 88, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" + } +}, { + "id" : "driver", + "hostPort" : "172.22.0.167:51475", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 384093388, + "executorLogs" : { } +}, { + "id" : "1", + "hostPort" : "172.22.0.167:51490", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 4, + "totalTasks" : 4, + "totalDuration" : 3152, + "totalGCTime" : 68, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout", + "stderr" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr" + } +}, { + "id" : "0", + "hostPort" : "172.22.0.167:51491", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2551, + "totalGCTime" : 116, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr" + } +}, { + "id" : "3", + "hostPort" : "172.22.0.167:51485", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2453, + "totalGCTime" : 72, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" + } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json new file mode 100644 index 0000000000000..92e249c851116 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -0,0 +1,118 @@ +[ { + "id" : "2", + "hostPort" : "172.22.0.111:64539", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 6, + "completedTasks" : 0, + "totalTasks" : 6, + "totalDuration" : 2792, + "totalGCTime" : 128, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + } +}, { + "id" : "driver", + "hostPort" : "172.22.0.111:64527", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { } +}, { + "id" : "1", + "hostPort" : "172.22.0.111:64541", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2613, + "totalGCTime" : 84, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout", + "stderr" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr" + } +}, { + "id" : "0", + "hostPort" : "172.22.0.111:64540", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2741, + "totalGCTime" : 120, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" + } +}, { + "id" : "3", + "hostPort" : "172.22.0.111:64543", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 4, + "totalTasks" : 4, + "totalDuration" : 3457, + "totalGCTime" : 72, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" + } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json index 9165f549d7d25..8820c717f85d9 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,67 +1,43 @@ [ { - "id" : "local-1430917381534", + "id" : "app-20161116163331-0000", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-05-06T13:03:11.398GMT", + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", "lastUpdated" : "", - "duration" : 10505, - "sparkUser" : "irashid", + "duration" : 10671, + "sparkUser" : "jose", "completed" : true, - "startTimeEpoch" : 1430917380893, - "endTimeEpoch" : 1430917391398, + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, "lastUpdatedEpoch" : 0 } ] }, { - "id" : "local-1430917381535", + "id" : "app-20161115172038-0000", "name" : "Spark shell", "attempts" : [ { - "attemptId" : "2", - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-05-06T13:03:00.950GMT", - "lastUpdated" : "", - "duration" : 57, - "sparkUser" : "irashid", - "completed" : true, - "startTimeEpoch" : 1430917380893, - "endTimeEpoch" : 1430917380950, - "lastUpdatedEpoch" : 0 - }, { - "attemptId" : "1", - "startTime" : "2015-05-06T13:03:00.880GMT", - "endTime" : "2015-05-06T13:03:00.890GMT", + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", "lastUpdated" : "", - "duration" : 10, - "sparkUser" : "irashid", + "duration" : 101795, + "sparkUser" : "jose", "completed" : true, - "startTimeEpoch" : 1430917380880, - "endTimeEpoch" : 1430917380890, + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, "lastUpdatedEpoch" : 0 } ] }, { - "id" : "local-1426533911241", + "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { - "attemptId" : "2", - "startTime" : "2015-03-17T23:11:50.242GMT", - "endTime" : "2015-03-17T23:12:25.177GMT", - "lastUpdated" : "", - "duration" : 34935, - "sparkUser" : "irashid", - "completed" : true, - "startTimeEpoch" : 1426633910242, - "endTimeEpoch" : 1426633945177, - "lastUpdatedEpoch" : 0 - }, { - "attemptId" : "1", - "startTime" : "2015-03-16T19:25:10.242GMT", - "endTime" : "2015-03-16T19:25:45.177GMT", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:11.398GMT", "lastUpdated" : "", - "duration" : 34935, + "duration" : 10505, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, - "endTimeEpoch" : 1426533945177, + "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json index a525d61543a88..c3fe4db222ae6 100644 --- a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json @@ -8,8 +8,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json index cc567f66f02e8..8281fa75aa0db 100644 --- a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json @@ -8,8 +8,8 @@ "duration" : 9011, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981779720, "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, "lastUpdatedEpoch" : 0 } ] }, { @@ -22,8 +22,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/maxEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxEndDate_app_list_json_expectation.json new file mode 100644 index 0000000000000..1842f1888b78c --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/maxEndDate_app_list_json_expectation.json @@ -0,0 +1,95 @@ +[ { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950 + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890 + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177 + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177 + } ] +}, { + "id" : "local-1425081759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-28T00:02:38.277GMT", + "endTime" : "2015-02-28T00:02:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1425081758277, + "endTimeEpoch" : 1425081766912 + } ] +}, { + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "lastUpdated" : "", + "duration" : 9011, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1422981779720, + "endTimeEpoch" : 1422981788731 + } ] +}, { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1422981758277, + "endTimeEpoch" : 1422981766912 + } ] +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_and_maxEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_and_maxEndDate_app_list_json_expectation.json new file mode 100644 index 0000000000000..24f9f21ec6501 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/minDate_and_maxEndDate_app_list_json_expectation.json @@ -0,0 +1,53 @@ +[ { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950 + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890 + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177 + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177 + } ] +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index c934a871724b5..1930281f1a3ec 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,4 +1,32 @@ [ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { @@ -8,8 +36,8 @@ "duration" : 10505, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 } ] }, { @@ -23,8 +51,8 @@ "duration" : 57, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917380950, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -34,8 +62,8 @@ "duration" : 10, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890, + "startTimeEpoch" : 1430917380880, "lastUpdatedEpoch" : 0 } ] }, { @@ -49,8 +77,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426633910242, "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -60,8 +88,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, "lastUpdatedEpoch" : 0 } ] }, { @@ -74,8 +102,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1425081758277, "endTimeEpoch" : 1425081766912, + "startTimeEpoch" : 1425081758277, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_and_maxEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_and_maxEndDate_app_list_json_expectation.json new file mode 100644 index 0000000000000..3745e8a09a98b --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_and_maxEndDate_app_list_json_expectation.json @@ -0,0 +1,53 @@ +[ { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950 + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890 + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177 + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177 + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json new file mode 100644 index 0000000000000..05233db441eda --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -0,0 +1,70 @@ +[ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0, + "endTimeEpoch" : 1479335620587 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0, + "endTimeEpoch" : 1479252138874 + } ] +}, { + "id" : "local-1430917381534", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:11.398GMT", + "lastUpdated" : "", + "duration" : 10505, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917391398 + } ] +}, { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, + "sparkUser" : "irashid", + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950 + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, + "sparkUser" : "irashid", + "completed" : true, + "completed" : true, + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890 + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json index f486d46313d8b..e8ed96dc85f09 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json @@ -8,8 +8,8 @@ "duration" : 9011, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981779720, "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, "lastUpdatedEpoch" : 0 } ] } diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json index e63039f6a17fc..88c601512d79f 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json @@ -9,8 +9,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426633910242, "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -20,8 +20,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, "lastUpdatedEpoch" : 0 } ] } diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index 0084339d24642..c2f450ba87c6d 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -30,8 +30,10 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.829GMT", + "duration" : 435, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -68,24 +70,26 @@ } } }, - "11" : { - "taskId" : 11, - "index" : 3, + "9" : { + "taskId" : 9, + "index" : 1, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 436, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 1, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 434, + "executorRunTime" : 436, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -105,19 +109,21 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, + "bytesWritten" : 1648, + "writeTime" : 98000, "recordsWritten" : 0 } } }, - "14" : { - "taskId" : 14, - "index" : 6, + "10" : { + "taskId" : 10, + "index" : 2, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -149,18 +155,20 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 88000, + "writeTime" : 76000, "recordsWritten" : 0 } } }, - "13" : { - "taskId" : 13, - "index" : 5, + "11" : { + "taskId" : 11, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -171,7 +179,7 @@ "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -191,19 +199,21 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, + "bytesWritten" : 1647, + "writeTime" : 83000, "recordsWritten" : 0 } } }, - "10" : { - "taskId" : 10, - "index" : 2, + "12" : { + "taskId" : 12, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -234,30 +244,32 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 76000, + "bytesWritten" : 1645, + "writeTime" : 101000, "recordsWritten" : 0 } } }, - "9" : { - "taskId" : 9, - "index" : 1, + "13" : { + "taskId" : 13, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, + "executorDeserializeTime" : 2, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 436, + "executorRunTime" : 434, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -278,18 +290,20 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 98000, + "writeTime" : 73000, "recordsWritten" : 0 } } }, - "12" : { - "taskId" : 12, - "index" : 4, + "14" : { + "taskId" : 14, + "index" : 6, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.832GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -320,8 +334,8 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1645, - "writeTime" : 101000, + "bytesWritten" : 1648, + "writeTime" : 88000, "recordsWritten" : 0 } } @@ -331,8 +345,10 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", + "duration" : 435, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index 63fe3b2f958e5..506859ae545b1 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -30,8 +30,10 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.829GMT", + "duration" : 435, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -68,24 +70,26 @@ } } }, - "11" : { - "taskId" : 11, - "index" : 3, + "9" : { + "taskId" : 9, + "index" : 1, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 436, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 1, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 434, + "executorRunTime" : 436, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -105,19 +109,21 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, + "bytesWritten" : 1648, + "writeTime" : 98000, "recordsWritten" : 0 } } }, - "14" : { - "taskId" : 14, - "index" : 6, + "10" : { + "taskId" : 10, + "index" : 2, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -149,18 +155,20 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 88000, + "writeTime" : 76000, "recordsWritten" : 0 } } }, - "13" : { - "taskId" : 13, - "index" : 5, + "11" : { + "taskId" : 11, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -171,7 +179,7 @@ "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -191,19 +199,21 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, + "bytesWritten" : 1647, + "writeTime" : 83000, "recordsWritten" : 0 } } }, - "10" : { - "taskId" : 10, - "index" : 2, + "12" : { + "taskId" : 12, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -234,30 +244,32 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 76000, + "bytesWritten" : 1645, + "writeTime" : 101000, "recordsWritten" : 0 } } }, - "9" : { - "taskId" : 9, - "index" : 1, + "13" : { + "taskId" : 13, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, + "executorDeserializeTime" : 2, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 436, + "executorRunTime" : 434, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -278,18 +290,20 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 98000, + "writeTime" : 73000, "recordsWritten" : 0 } } }, - "12" : { - "taskId" : 12, - "index" : 4, + "14" : { + "taskId" : 14, + "index" : 6, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.832GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -320,8 +334,8 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1645, - "writeTime" : 101000, + "bytesWritten" : 1648, + "writeTime" : 88000, "recordsWritten" : 0 } } @@ -331,8 +345,10 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", + "duration" : 435, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json index e0661c464179d..f4cec68fbfdf2 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json @@ -3,8 +3,10 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.494GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -45,8 +47,10 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.502GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -87,8 +91,10 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", + "duration" : 348, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -129,8 +135,10 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -171,8 +179,10 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -213,8 +223,10 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -255,8 +267,10 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 351, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -297,8 +311,10 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.506GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -339,8 +355,10 @@ "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", + "duration" : 80, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -381,8 +399,10 @@ "index" : 9, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.915GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -423,8 +443,10 @@ "index" : 10, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.916GMT", + "duration" : 73, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -465,8 +487,10 @@ "index" : 11, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.918GMT", + "duration" : 75, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -507,8 +531,10 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", + "duration" : 77, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -549,8 +575,10 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -591,8 +619,10 @@ "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -633,8 +663,10 @@ "index" : 15, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.928GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -675,8 +707,10 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -717,8 +751,10 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", + "duration" : 91, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -759,8 +795,10 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", + "duration" : 92, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -801,8 +839,10 @@ "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json index 8492f19ab7a5f..496a21c328da9 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json @@ -3,8 +3,10 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.515GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -50,8 +52,10 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.521GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -97,8 +101,10 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -144,8 +150,10 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -191,8 +199,10 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -238,8 +248,10 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -285,8 +297,10 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -332,8 +346,10 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.524GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json index 4de4c501a43ad..4328dc753c5d4 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json @@ -3,8 +3,10 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.515GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -50,8 +52,10 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.521GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -97,8 +101,10 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -144,8 +150,10 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -191,8 +199,10 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -238,8 +248,10 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -285,8 +297,10 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -332,8 +346,10 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.524GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json index d2eceeb3f97a9..8c571430f3a1f 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json @@ -3,8 +3,10 @@ "index" : 10, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.916GMT", + "duration" : 73, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -45,8 +47,10 @@ "index" : 11, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.918GMT", + "duration" : 75, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -87,8 +91,10 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", + "duration" : 77, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -129,8 +135,10 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -171,8 +179,10 @@ "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -213,8 +223,10 @@ "index" : 15, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.928GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -255,8 +267,10 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -297,8 +311,10 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", + "duration" : 91, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -339,8 +355,10 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", + "duration" : 92, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -381,8 +399,10 @@ "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -423,8 +443,10 @@ "index" : 20, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.014GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -465,8 +487,10 @@ "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", + "duration" : 88, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -507,8 +531,10 @@ "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", + "duration" : 93, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -549,8 +575,10 @@ "index" : 23, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.031GMT", + "duration" : 65, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -591,8 +619,10 @@ "index" : 24, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.098GMT", + "duration" : 43, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -633,8 +663,10 @@ "index" : 25, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.103GMT", + "duration" : 49, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -675,8 +707,10 @@ "index" : 26, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.105GMT", + "duration" : 38, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -717,8 +751,10 @@ "index" : 27, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.110GMT", + "duration" : 32, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -759,8 +795,10 @@ "index" : 28, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.113GMT", + "duration" : 29, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -801,8 +839,10 @@ "index" : 29, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.114GMT", + "duration" : 39, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -843,8 +883,10 @@ "index" : 30, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.118GMT", + "duration" : 34, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -885,8 +927,10 @@ "index" : 31, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.127GMT", + "duration" : 24, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -927,8 +971,10 @@ "index" : 32, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.148GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -969,8 +1015,10 @@ "index" : 33, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.149GMT", + "duration" : 43, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1011,8 +1059,10 @@ "index" : 34, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.156GMT", + "duration" : 27, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1053,8 +1103,10 @@ "index" : 35, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.161GMT", + "duration" : 35, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1095,8 +1147,10 @@ "index" : 36, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.164GMT", + "duration" : 29, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1137,8 +1191,10 @@ "index" : 37, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.165GMT", + "duration" : 32, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1179,8 +1235,10 @@ "index" : 38, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.166GMT", + "duration" : 31, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1221,8 +1279,10 @@ "index" : 39, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.180GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1263,8 +1323,10 @@ "index" : 40, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.197GMT", + "duration" : 14, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1305,8 +1367,10 @@ "index" : 41, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.200GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1347,8 +1411,10 @@ "index" : 42, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.203GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1389,8 +1455,10 @@ "index" : 43, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.204GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1431,8 +1499,10 @@ "index" : 44, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.205GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1473,8 +1543,10 @@ "index" : 45, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.206GMT", + "duration" : 19, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1515,8 +1587,10 @@ "index" : 46, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.210GMT", + "duration" : 31, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1557,8 +1631,10 @@ "index" : 47, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.212GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1599,8 +1675,10 @@ "index" : 48, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.220GMT", + "duration" : 24, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1641,8 +1719,10 @@ "index" : 49, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.223GMT", + "duration" : 23, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1683,8 +1763,10 @@ "index" : 50, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.240GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1725,8 +1807,10 @@ "index" : 51, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.242GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1767,8 +1851,10 @@ "index" : 52, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.243GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1809,8 +1895,10 @@ "index" : 53, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.244GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1851,8 +1939,10 @@ "index" : 54, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.244GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1893,8 +1983,10 @@ "index" : 55, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.246GMT", + "duration" : 21, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1935,8 +2027,10 @@ "index" : 56, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.249GMT", + "duration" : 20, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -1977,8 +2071,10 @@ "index" : 57, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.257GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -2019,8 +2115,10 @@ "index" : 58, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.263GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -2061,8 +2159,10 @@ "index" : 59, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.265GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json index f42c3a4ee5c38..0bd614bdc756e 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json @@ -3,8 +3,10 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 351, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -45,8 +47,10 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.502GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -87,8 +91,10 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -129,8 +135,10 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.494GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -171,8 +179,10 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -213,8 +223,10 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -255,8 +267,10 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.506GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -297,8 +311,10 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", + "duration" : 348, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -339,8 +355,10 @@ "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", + "duration" : 93, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -381,8 +399,10 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", + "duration" : 92, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -423,8 +443,10 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", + "duration" : 91, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -465,8 +487,10 @@ "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", + "duration" : 88, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -507,8 +531,10 @@ "index" : 9, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.915GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -549,8 +575,10 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -591,8 +619,10 @@ "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -633,8 +663,10 @@ "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -675,8 +707,10 @@ "index" : 20, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.014GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -717,8 +751,10 @@ "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", + "duration" : 80, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -759,8 +795,10 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", + "duration" : 77, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -801,8 +839,10 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json index f42c3a4ee5c38..0bd614bdc756e 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json @@ -3,8 +3,10 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 351, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -45,8 +47,10 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.502GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -87,8 +91,10 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -129,8 +135,10 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.494GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -171,8 +179,10 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -213,8 +223,10 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -255,8 +267,10 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.506GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -297,8 +311,10 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", + "duration" : 348, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -339,8 +355,10 @@ "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", + "duration" : 93, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -381,8 +399,10 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", + "duration" : 92, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -423,8 +443,10 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", + "duration" : 91, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -465,8 +487,10 @@ "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", + "duration" : 88, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -507,8 +531,10 @@ "index" : 9, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.915GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -549,8 +575,10 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -591,8 +619,10 @@ "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -633,8 +663,10 @@ "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -675,8 +707,10 @@ "index" : 20, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.014GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -717,8 +751,10 @@ "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", + "duration" : 80, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -759,8 +795,10 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", + "duration" : 77, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -801,8 +839,10 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json index db60ccccbf8c8..b58f1a51ba481 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json @@ -3,8 +3,10 @@ "index" : 40, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.197GMT", + "duration" : 14, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -45,8 +47,10 @@ "index" : 41, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.200GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -87,8 +91,10 @@ "index" : 43, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.204GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -129,8 +135,10 @@ "index" : 57, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.257GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -171,8 +179,10 @@ "index" : 58, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.263GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -213,8 +223,10 @@ "index" : 68, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.306GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -255,8 +267,10 @@ "index" : 86, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.374GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -297,8 +311,10 @@ "index" : 32, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.148GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -339,8 +355,10 @@ "index" : 39, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.180GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -381,8 +399,10 @@ "index" : 42, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.203GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -423,8 +443,10 @@ "index" : 51, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.242GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -465,8 +487,10 @@ "index" : 59, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.265GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -507,8 +531,10 @@ "index" : 63, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.276GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -549,8 +575,10 @@ "index" : 87, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.374GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -591,8 +619,10 @@ "index" : 90, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.385GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -633,8 +663,10 @@ "index" : 99, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.426GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -675,8 +707,10 @@ "index" : 44, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.205GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -717,8 +751,10 @@ "index" : 47, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.212GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -759,8 +795,10 @@ "index" : 50, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.240GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], @@ -801,8 +839,10 @@ "index" : 52, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.243GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json index 5dcbc890438b2..0ed609d5b7f92 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json @@ -3,7 +3,7 @@ "executorDeserializeTime" : [ 1.0, 3.0, 36.0 ], "executorDeserializeCpuTime" : [ 0.0, 0.0, 0.0 ], "executorRunTime" : [ 16.0, 28.0, 351.0 ], - "executorCpuTime" : [ 0.0, 0.0, 0.0], + "executorCpuTime" : [ 0.0, 0.0, 0.0 ], "resultSize" : [ 2010.0, 2065.0, 2065.0 ], "jvmGcTime" : [ 0.0, 0.0, 7.0 ], "resultSerializationTime" : [ 0.0, 0.0, 2.0 ], diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index aaeef1f2f582c..a449926ee7dc6 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -29,23 +29,25 @@ "value" : "5050" } ], "tasks" : { - "2" : { - "taskId" : 2, - "index" : 2, + "0" : { + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", + "launchTime" : "2015-03-16T19:25:36.515GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "378", - "value" : "378" + "update" : "78", + "value" : "5050" } ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 14, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -77,23 +79,25 @@ } } }, - "5" : { - "taskId" : 5, - "index" : 5, + "1" : { + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.523GMT", + "launchTime" : "2015-03-16T19:25:36.521GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "897", - "value" : "3750" + "update" : "247", + "value" : "2175" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 14, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -125,29 +129,31 @@ } } }, - "4" : { - "taskId" : 4, - "index" : 4, + "2" : { + "taskId" : 2, + "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "678", - "value" : "2853" + "update" : "378", + "value" : "378" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 13, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -173,23 +179,25 @@ } } }, - "7" : { - "taskId" : 7, - "index" : 7, + "3" : { + "taskId" : 3, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.524GMT", + "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "1222", - "value" : "4972" + "update" : "572", + "value" : "950" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 13, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -221,29 +229,31 @@ } } }, - "1" : { - "taskId" : 1, - "index" : 1, + "4" : { + "taskId" : 4, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.521GMT", + "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "247", - "value" : "2175" + "update" : "678", + "value" : "2853" } ], "taskMetrics" : { - "executorDeserializeTime" : 14, + "executorDeserializeTime" : 12, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -269,23 +279,25 @@ } } }, - "3" : { - "taskId" : 3, - "index" : 3, + "5" : { + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", + "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "572", - "value" : "950" + "update" : "897", + "value" : "3750" } ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 12, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -322,8 +334,10 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -365,23 +379,25 @@ } } }, - "0" : { - "taskId" : 0, - "index" : 0, + "7" : { + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.515GMT", + "launchTime" : "2015-03-16T19:25:36.524GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "78", - "value" : "5050" + "update" : "1222", + "value" : "4972" } ], "taskMetrics" : { - "executorDeserializeTime" : 14, + "executorDeserializeTime" : 12, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, diff --git a/core/src/test/resources/spark-events/app-20161115172038-0000 b/core/src/test/resources/spark-events/app-20161115172038-0000 new file mode 100755 index 0000000000000..3af0451d0c392 --- /dev/null +++ b/core/src/test/resources/spark-events/app-20161115172038-0000 @@ -0,0 +1,75 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.1.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.111","Port":64527},"Maximum Memory":384093388,"Timestamp":1479252038836} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.111","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"64511","spark.repl.class.uri":"spark://172.22.0.111:64511/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-f09ef9e2-7f15-433f-a5d1-30138d8764ca/repl-28d60911-dbc3-465f-b7b3-ee55c071595e","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"10000","spark.app.id":"app-20161115172038-0000","spark.task.maxFailures":"4"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=10000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20161115172038-0000","Timestamp":1479252037079,"User":"jose"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042589,"Executor ID":"2","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout","stderr":"http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042593,"Executor ID":"0","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout","stderr":"http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042629,"Executor ID":"1","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout","stderr":"http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"172.22.0.111","Port":64540},"Maximum Memory":384093388,"Timestamp":1479252042687} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"172.22.0.111","Port":64539},"Maximum Memory":384093388,"Timestamp":1479252042689} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"172.22.0.111","Port":64541},"Maximum Memory":384093388,"Timestamp":1479252042692} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042711,"Executor ID":"3","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout","stderr":"http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.111","Port":64543},"Maximum Memory":384093388,"Timestamp":1479252042759} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1479252043855,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]}],"Stage IDs":[0],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479252044021,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479252044053,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479252044054,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479252044057,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479252044059,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479252044060,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479252044064,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479252044065,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479252044059,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044653,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":499,"Value":499,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":52390000,"Value":52390000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":18,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":7909000,"Value":7909000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":1123,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":21,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":499,"Executor Deserialize CPU Time":52390000,"Executor Run Time":18,"Executor CPU Time":7909000,"Result Size":1123,"JVM GC Time":21,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479252044054,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044657,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":508,"Value":1007,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":36827000,"Value":89217000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":18,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":4333000,"Value":12242000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":2246,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":42,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":508,"Executor Deserialize CPU Time":36827000,"Executor Run Time":18,"Executor CPU Time":4333000,"Result Size":1123,"JVM GC Time":21,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044658,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":509,"Value":1516,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":44100000,"Value":133317000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":17,"Value":53,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":11340000,"Value":23582000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":3369,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":509,"Executor Deserialize CPU Time":44100000,"Executor Run Time":17,"Executor CPU Time":11340000,"Result Size":1123,"JVM GC Time":21,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479252044021,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044692,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":511,"Value":2027,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":227762000,"Value":361079000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":3631000,"Value":27213000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1938,"Value":5307,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":511,"Executor Deserialize CPU Time":227762000,"Executor Run Time":16,"Executor CPU Time":3631000,"Result Size":1938,"JVM GC Time":21,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":495,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044720,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":495,"Value":564,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":114,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":495,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044727,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Value":1058,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":144,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":494,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479252044060,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044729,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Value":1552,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":174,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":494,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":13,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":5,"Attempt":1,"Launch Time":1479252044732,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044736,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Value":2003,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":206,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":451,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":11,"Attempt":1,"Launch Time":1479252044736,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479252044065,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044737,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Value":2449,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":238,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":446,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":15,"Attempt":1,"Launch Time":1479252044737,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":448,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044741,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":448,"Value":2897,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":270,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":448,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":7,"Attempt":1,"Launch Time":1479252044742,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044752,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044748,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":2035,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3655000,"Value":364734000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":899000,"Value":28112000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":884,"Value":6191,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":3655000,"Executor Run Time":0,"Executor CPU Time":899000,"Result Size":884,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":19,"Index":11,"Attempt":1,"Launch Time":1479252044736,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044749,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":2899,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":2,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":11,"Attempt":2,"Launch Time":1479252044749,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":7,"Attempt":1,"Launch Time":1479252044742,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044752,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":2038,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3566000,"Value":368300000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":2900,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1004000,"Value":29116000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":7154,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3566000,"Executor Run Time":1,"Executor CPU Time":1004000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":10,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":20,"Index":15,"Attempt":1,"Launch Time":1479252044737,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044756,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":10,"Value":2910,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":10,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":15,"Attempt":2,"Launch Time":1479252044756,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":11,"Attempt":2,"Launch Time":1479252044749,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044759,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":2042,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3720000,"Value":372020000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":2911,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1009000,"Value":30125000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":8117,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3720000,"Executor Run Time":1,"Executor CPU Time":1009000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":5,"Attempt":1,"Launch Time":1479252044732,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044760,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":2047,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4303000,"Value":376323000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":2913,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":999000,"Value":31124000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":9080,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4303000,"Executor Run Time":2,"Executor CPU Time":999000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":15,"Attempt":2,"Launch Time":1479252044756,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044768,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":2053,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4946000,"Value":381269000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":2914,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1176000,"Value":32300000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":10043,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4946000,"Executor Run Time":1,"Executor CPU Time":1176000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":13,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044775,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":2060,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3406000,"Value":384675000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1007000,"Value":33307000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":971,"Value":11014,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":3406000,"Executor Run Time":0,"Executor CPU Time":1007000,"Result Size":971,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":456,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479252044053,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044778,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":456,"Value":3370,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":302,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":456,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":3,"Attempt":1,"Launch Time":1479252044778,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":503,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479252044057,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044789,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":503,"Value":3873,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":332,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":503,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":9,"Attempt":1,"Launch Time":1479252044789,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":3,"Attempt":1,"Launch Time":1479252044778,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044791,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":2065,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2950000,"Value":387625000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3875,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":822000,"Value":34129000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":11977,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":2950000,"Executor Run Time":2,"Executor CPU Time":822000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":9,"Attempt":1,"Launch Time":1479252044789,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044798,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":2068,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2604000,"Value":390229000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3876,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":845000,"Value":34974000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":12940,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2604000,"Executor Run Time":1,"Executor CPU Time":845000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044920,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":784,"Value":2852,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":56180000,"Value":446409000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":24,"Value":3900,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":6046000,"Value":41020000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":13976,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":350,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":784,"Executor Deserialize CPU Time":56180000,"Executor Run Time":24,"Executor CPU Time":6046000,"Result Size":1036,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044921,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":789,"Value":3641,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":34766000,"Value":481175000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":22,"Value":3922,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":8189000,"Value":49209000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":15012,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":368,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":789,"Executor Deserialize CPU Time":34766000,"Executor Run Time":22,"Executor CPU Time":8189000,"Result Size":1036,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479252044064,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044921,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":777,"Value":4418,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":29960000,"Value":511135000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":24,"Value":3946,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":9708000,"Value":58917000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":16048,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":386,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":777,"Executor Deserialize CPU Time":29960000,"Executor Run Time":24,"Executor CPU Time":9708000,"Result Size":1036,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044924,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":791,"Value":5209,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":266560000,"Value":777695000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":3962,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":5884000,"Value":64801000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1851,"Value":17899,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":404,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":791,"Executor Deserialize CPU Time":266560000,"Executor Run Time":16,"Executor CPU Time":5884000,"Result Size":1851,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Submission Time":1479252044017,"Completion Time":1479252044926,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":3962,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":404,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":17899,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":777695000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":64801000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":5209,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1479252044931,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479252044930,"executorId":"2","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479252044930,"executorId":"0","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerNodeBlacklisted","time":1479252044930,"hostId":"172.22.0.111","executorFailures":2} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted","time":1479252055635,"executorId":"2"} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted","time":1479252055635,"executorId":"0"} +{"Event":"org.apache.spark.scheduler.SparkListenerNodeUnblacklisted","time":1479252055635,"hostId":"172.22.0.111"} +{"Event":"SparkListenerApplicationEnd","Timestamp":1479252138874} diff --git a/core/src/test/resources/spark-events/app-20161116163331-0000 b/core/src/test/resources/spark-events/app-20161116163331-0000 new file mode 100755 index 0000000000000..7566c9fc0a20b --- /dev/null +++ b/core/src/test/resources/spark-events/app-20161116163331-0000 @@ -0,0 +1,68 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.1.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.167","Port":51475},"Maximum Memory":384093388,"Timestamp":1479335611477} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.167","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"51459","spark.repl.class.uri":"spark://172.22.0.167:51459/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"1000000","spark.app.id":"app-20161116163331-0000","spark.task.maxFailures":"4"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20161116163331-0000","Timestamp":1479335609916,"User":"jose"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615320,"Executor ID":"3","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout","stderr":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.167","Port":51485},"Maximum Memory":384093388,"Timestamp":1479335615387} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615393,"Executor ID":"2","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout","stderr":"http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615443,"Executor ID":"1","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout","stderr":"http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"172.22.0.167","Port":51487},"Maximum Memory":384093388,"Timestamp":1479335615448} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615462,"Executor ID":"0","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout","stderr":"http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"172.22.0.167","Port":51490},"Maximum Memory":384093388,"Timestamp":1479335615496} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"172.22.0.167","Port":51491},"Maximum Memory":384093388,"Timestamp":1479335615515} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1479335616467,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]}],"Stage IDs":[0],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479335616657,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479335616687,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479335616689,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479335616690,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479335616691,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479335616693,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479335616695,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479335616697,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617253,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":465,"Value":465,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":47305000,"Value":47305000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":22,"Value":22,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":7220000,"Value":7220000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":1123,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":465,"Executor Deserialize CPU Time":47305000,"Executor Run Time":22,"Executor CPU Time":7220000,"Result Size":1123,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479335616697,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617257,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":464,"Value":929,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":20082000,"Value":67387000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":21,"Value":43,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":9084000,"Value":16304000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":2246,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":464,"Executor Deserialize CPU Time":20082000,"Executor Run Time":21,"Executor CPU Time":9084000,"Result Size":1123,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617257,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":468,"Value":1397,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":29183000,"Value":96570000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":21,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":5753000,"Value":22057000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":3369,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":54,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":468,"Executor Deserialize CPU Time":29183000,"Executor Run Time":21,"Executor CPU Time":5753000,"Result Size":1123,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617257,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":470,"Value":1867,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":233387000,"Value":329957000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":22,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":6783000,"Value":28840000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1938,"Value":5307,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":470,"Executor Deserialize CPU Time":233387000,"Executor Run Time":22,"Executor CPU Time":6783000,"Result Size":1938,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":453,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479335616690,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617319,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":453,"Value":539,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":94,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":453,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617326,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Value":983,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":123,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":444,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479335616687,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617327,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Value":1434,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":145,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":451,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617328,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Value":1885,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":167,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":451,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":450,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479335616693,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617329,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":450,"Value":2335,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":189,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":450,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617329,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Value":2779,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":218,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":444,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":442,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617329,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":442,"Value":3221,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":247,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":442,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":2,"Attempt":1,"Launch Time":1479335617332,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":1903,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5136000,"Value":346556000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3673,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":958000,"Value":32856000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":9159,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":10,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617370,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":1889,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3808000,"Value":341420000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3672,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1005000,"Value":31898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":8196,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":9,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617369,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":1879,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3737000,"Value":337612000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3670,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1066000,"Value":30893000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":7233,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":13,"Attempt":1,"Launch Time":1479335617334,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617368,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":1872,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3918000,"Value":333875000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3668,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":987000,"Value":29827000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":6270,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479335616691,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617336,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Value":3667,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":276,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":446,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":6,"Attempt":1,"Launch Time":1479335617349,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1907,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3503000,"Value":350059000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3674,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1042000,"Value":33898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":10122,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":13,"Attempt":1,"Launch Time":1479335617334,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617368,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":1872,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3918000,"Value":333875000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3668,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":987000,"Value":29827000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":6270,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3918000,"Executor Run Time":1,"Executor CPU Time":987000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":1,"Attempt":1,"Launch Time":1479335617368,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617379,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1911,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3579000,"Value":353638000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3675,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":996000,"Value":34894000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":11085,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":9,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617369,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":1879,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3737000,"Value":337612000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3670,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1066000,"Value":30893000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":7233,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":3737000,"Executor Run Time":2,"Executor CPU Time":1066000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":14,"Attempt":1,"Launch Time":1479335617369,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1915,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3412000,"Value":357050000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3676,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1014000,"Value":35908000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":12048,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":10,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617370,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":1889,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3808000,"Value":341420000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3672,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1005000,"Value":31898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":8196,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":3808000,"Executor Run Time":2,"Executor CPU Time":1005000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":5,"Attempt":1,"Launch Time":1479335617370,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":1918,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3482000,"Value":360532000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3678,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1142000,"Value":37050000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":13011,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":2,"Attempt":1,"Launch Time":1479335617332,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":1903,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5136000,"Value":346556000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3673,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":958000,"Value":32856000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":9159,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":5136000,"Executor Run Time":1,"Executor CPU Time":958000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":6,"Attempt":1,"Launch Time":1479335617349,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1907,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3503000,"Value":350059000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3674,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1042000,"Value":33898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":10122,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3503000,"Executor Run Time":1,"Executor CPU Time":1042000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":1,"Attempt":1,"Launch Time":1479335617368,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617379,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1911,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3579000,"Value":353638000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3675,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":996000,"Value":34894000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":11085,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3579000,"Executor Run Time":1,"Executor CPU Time":996000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":14,"Attempt":1,"Launch Time":1479335617369,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1915,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3412000,"Value":357050000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3676,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1014000,"Value":35908000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":12048,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3412000,"Executor Run Time":1,"Executor CPU Time":1014000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":5,"Attempt":1,"Launch Time":1479335617370,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":1918,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3482000,"Value":360532000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3678,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1142000,"Value":37050000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":13011,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3482000,"Executor Run Time":2,"Executor CPU Time":1142000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479335616657,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617470,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":714,"Value":2632,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":41300000,"Value":401832000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":18,"Value":3696,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":6640000,"Value":43690000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":14047,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":293,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":714,"Executor Deserialize CPU Time":41300000,"Executor Run Time":18,"Executor CPU Time":6640000,"Result Size":1036,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617471,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":714,"Value":3346,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":43682000,"Value":445514000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":15,"Value":3711,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":9441000,"Value":53131000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":15083,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":310,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":714,"Executor Deserialize CPU Time":43682000,"Executor Run Time":15,"Executor CPU Time":9441000,"Result Size":1036,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479335616695,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617471,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":691,"Value":4037,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54811000,"Value":500325000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":3727,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":4571000,"Value":57702000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":16119,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":327,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":691,"Executor Deserialize CPU Time":54811000,"Executor Run Time":16,"Executor CPU Time":4571000,"Result Size":1036,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479335616689,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617473,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":716,"Value":4753,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":220235000,"Value":720560000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":3743,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":5849000,"Value":63551000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1851,"Value":17970,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":344,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":716,"Executor Deserialize CPU Time":220235000,"Executor Run Time":16,"Executor CPU Time":5849000,"Result Size":1851,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Submission Time":1479335616653,"Completion Time":1479335617476,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":3743,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":344,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":17970,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":720560000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":63551000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":4,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":4753,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1479335617480,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479335617478,"executorId":"2","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479335617478,"executorId":"0","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerNodeBlacklisted","time":1479335617478,"hostId":"172.22.0.167","executorFailures":2} +{"Event":"SparkListenerApplicationEnd","Timestamp":1479335620587} diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 840f55ce2f6e5..8d7be77f51fe9 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark -import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 83906cff123bf..0897891ee1758 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -303,6 +303,25 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } } + test("encryption requires authentication") { + val conf = new SparkConf() + conf.validateSettings() + + conf.set(NETWORK_ENCRYPTION_ENABLED, true) + intercept[IllegalArgumentException] { + conf.validateSettings() + } + + conf.set(NETWORK_ENCRYPTION_ENABLED, false) + conf.set(SASL_ENCRYPTION_ENABLED, true) + intercept[IllegalArgumentException] { + conf.validateSettings() + } + + conf.set(NETWORK_AUTH_ENABLED, true) + conf.validateSettings() + } + } class Class1 {} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index d3b79dd3e36e8..b2eded43ba71f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -102,6 +102,12 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "minDate app list json" -> "applications?minDate=2015-02-10", "maxDate app list json" -> "applications?maxDate=2015-02-10", "maxDate2 app list json" -> "applications?maxDate=2015-02-03T16:42:40.000GMT", + "minEndDate app list json" -> "applications?minEndDate=2015-05-06T13:03:00.950GMT", + "maxEndDate app list json" -> "applications?maxEndDate=2015-05-06T13:03:00.950GMT", + "minEndDate and maxEndDate app list json" -> + "applications?minEndDate=2015-03-16&maxEndDate=2015-05-06T13:03:00.950GMT", + "minDate and maxEndDate app list json" -> + "applications?minDate=2015-03-16&maxEndDate=2015-05-06T13:03:00.950GMT", "limit app list json" -> "applications?limit=3", "one app json" -> "applications/local-1422981780767", "one app multi-attempt json" -> "applications/local-1426533911241", @@ -143,7 +149,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "stage task list from multi-attempt app json(2)" -> "applications/local-1426533911241/2/stages/0/0/taskList", - "rdd list storage json" -> "applications/local-1422981780767/storage/rdd" + "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", + "executor node blacklisting" -> "applications/app-20161116163331-0000/executors", + "executor node blacklisting unblacklisting" -> "applications/app-20161115172038-0000/executors" // Todo: enable this test when logging the even of onBlockUpdated. See: SPARK-13845 // "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" ) diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 022fe91edade9..fe8955840d72f 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -94,6 +94,20 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi } } + test("security with aes encryption") { + val conf = new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.app.id", "app-id") + .set("spark.network.crypto.enabled", "true") + .set("spark.network.crypto.saslFallback", "false") + testConnection(conf, conf) match { + case Success(_) => // expected + case Failure(t) => fail(t) + } + } + + /** * Creates two servers with different configurations and sees if they can talk. * Returns Success() if they can transfer a block, and Failure() if the block transfer was failed diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala index 121447a96529b..271ab8b148831 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala @@ -23,7 +23,6 @@ import org.mockito.Mockito.mock import org.scalatest._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.internal.config._ import org.apache.spark.network.BlockDataManager class NettyBlockTransferServiceSuite diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index acdf21df9a161..b4037d7a9c6e8 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -637,11 +637,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { assert(anotherEnv.address.port != env.address.port) } - test("send with authentication") { - val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - + private def testSend(conf: SparkConf): Unit = { val localEnv = createRpcEnv(conf, "authentication-local", 0) val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true) @@ -667,11 +663,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } } - test("ask with authentication") { - val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - + private def testAsk(conf: SparkConf): Unit = { val localEnv = createRpcEnv(conf, "authentication-local", 0) val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true) @@ -695,6 +687,48 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } } + test("send with authentication") { + testSend(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good")) + } + + test("send with SASL encryption") { + testSend(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.authenticate.enableSaslEncryption", "true")) + } + + test("send with AES encryption") { + testSend(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.network.crypto.enabled", "true") + .set("spark.network.crypto.saslFallback", "false")) + } + + test("ask with authentication") { + testAsk(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good")) + } + + test("ask with SASL encryption") { + testAsk(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.authenticate.enableSaslEncryption", "true")) + } + + test("ask with AES encryption") { + testAsk(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.network.crypto.enabled", "true") + .set("spark.network.crypto.saslFallback", "false")) + } + test("construct RpcTimeout with conf property") { val conf = new SparkConf diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala index 0409aa3a5dee1..2b1bce4d208f6 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.rpc.netty +import org.scalatest.mock.MockitoSugar + import org.apache.spark._ +import org.apache.spark.network.client.TransportClient import org.apache.spark.rpc._ -class NettyRpcEnvSuite extends RpcEnvSuite { +class NettyRpcEnvSuite extends RpcEnvSuite with MockitoSugar { override def createRpcEnv( conf: SparkConf, @@ -53,4 +56,32 @@ class NettyRpcEnvSuite extends RpcEnvSuite { } } + test("RequestMessage serialization") { + def assertRequestMessageEquals(expected: RequestMessage, actual: RequestMessage): Unit = { + assert(expected.senderAddress === actual.senderAddress) + assert(expected.receiver === actual.receiver) + assert(expected.content === actual.content) + } + + val nettyEnv = env.asInstanceOf[NettyRpcEnv] + val client = mock[TransportClient] + val senderAddress = RpcAddress("locahost", 12345) + val receiverAddress = RpcEndpointAddress("localhost", 54321, "test") + val receiver = new NettyRpcEndpointRef(nettyEnv.conf, receiverAddress, nettyEnv) + + val msg = new RequestMessage(senderAddress, receiver, "foo") + assertRequestMessageEquals( + msg, + RequestMessage(nettyEnv, client, msg.serialize(nettyEnv))) + + val msg2 = new RequestMessage(null, receiver, "foo") + assertRequestMessageEquals( + msg2, + RequestMessage(nettyEnv, client, msg2.serialize(nettyEnv))) + + val msg3 = new RequestMessage(senderAddress, receiver, null) + assertRequestMessageEquals( + msg3, + RequestMessage(nettyEnv, client, msg3.serialize(nettyEnv))) + } } diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala index 0c156fef0ae0f..a71d8726e7066 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala @@ -34,7 +34,7 @@ class NettyRpcHandlerSuite extends SparkFunSuite { val env = mock(classOf[NettyRpcEnv]) val sm = mock(classOf[StreamManager]) when(env.deserialize(any(classOf[TransportClient]), any(classOf[ByteBuffer]))(any())) - .thenReturn(RequestMessage(RpcAddress("localhost", 12345), null, null)) + .thenReturn(new RequestMessage(RpcAddress("localhost", 12345), null, null)) test("receive") { val dispatcher = mock(classOf[Dispatcher]) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 6b314d2ae339f..ead695574d292 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.mockito.Mockito.when +import org.mockito.Mockito.{verify, when} import org.scalatest.BeforeAndAfterEach import org.scalatest.mock.MockitoSugar @@ -31,6 +31,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M private val clock = new ManualClock(0) private var blacklist: BlacklistTracker = _ + private var listenerBusMock: LiveListenerBus = _ private var scheduler: TaskSchedulerImpl = _ private var conf: SparkConf = _ @@ -40,7 +41,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M scheduler = mockTaskSchedWithConf(conf) clock.setTime(0) - blacklist = new BlacklistTracker(conf, clock) + + listenerBusMock = mock[LiveListenerBus] + blacklist = new BlacklistTracker(listenerBusMock, conf, clock) } override def afterEach(): Unit = { @@ -112,6 +115,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) } else { assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post( + SparkListenerExecutorBlacklisted(0, "1", failuresUntilBlacklisted)) } } } @@ -147,6 +152,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // and it should be blacklisted for the entire application. blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", numFailures)) } else { // The task set failed, so we don't count these failures against the executor for other // stages. @@ -166,6 +172,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assert(blacklist.nodeBlacklist() === Set()) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 4)) val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) // Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole @@ -177,15 +184,21 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures) assert(blacklist.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(0, "hostA", 2)) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 4)) // Advance the clock and then make sure hostA and executors 1 and 2 have been removed from the // blacklist. - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) + val timeout = blacklist.BLACKLIST_TIMEOUT_MILLIS + 1 + clock.advance(timeout) blacklist.applyBlacklistTimeout() assert(blacklist.nodeBlacklist() === Set()) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "2")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "1")) + verify(listenerBusMock).post(SparkListenerNodeUnblacklisted(timeout, "hostA")) // Fail one more task, but executor isn't put back into blacklist since the count of failures // on that executor should have been reset to 0. @@ -212,7 +225,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M stageAttemptId = 0, taskSetBlacklist0.execToFailures) assert(blacklist.isExecutorBlacklisted("1")) - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 4)) + val t1 = blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + clock.advance(t1) // Now another executor gets spun up on that host, but it also dies. val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) @@ -227,22 +242,29 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // We've now had two bad executors on the hostA, so we should blacklist the entire node. assert(blacklist.isExecutorBlacklisted("1")) assert(blacklist.isExecutorBlacklisted("2")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t1, "2", 4)) assert(blacklist.isNodeBlacklisted("hostA")) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(t1, "hostA", 2)) // Advance the clock so that executor 1 should no longer be explicitly blacklisted, but // everything else should still be blacklisted. - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + 1) + val t2 = blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + 1 + clock.advance(t2) blacklist.applyBlacklistTimeout() assert(!blacklist.isExecutorBlacklisted("1")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(t1 + t2, "1")) assert(blacklist.isExecutorBlacklisted("2")) assert(blacklist.isNodeBlacklisted("hostA")) // make sure we don't leak memory assert(!blacklist.executorIdToBlacklistStatus.contains("1")) assert(!blacklist.nodeToBlacklistedExecs("hostA").contains("1")) // Advance the timeout again so now hostA should be removed from the blacklist. - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2) + clock.advance(t1) blacklist.applyBlacklistTimeout() assert(!blacklist.nodeIdToBlacklistExpiryTime.contains("hostA")) + verify(listenerBusMock).post(SparkListenerNodeUnblacklisted(t1 + t2 + t1, "hostA")) + // Even though unblacklisting a node implicitly unblacklists all of its executors, + // there will be no SparkListenerExecutorUnblacklisted sent here. } test("task failures expire with time") { @@ -280,6 +302,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M failOneTaskInTaskSet(exec = "1") blacklist.applyBlacklistTimeout() assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t1, "1", 2)) assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) // Add failures on executor 3, make sure it gets put on the blacklist. @@ -289,12 +312,14 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M failOneTaskInTaskSet(exec = "3") blacklist.applyBlacklistTimeout() assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "3")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t2, "3", 2)) assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) // Now we go past the timeout for executor 1, so it should be dropped from the blacklist. clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) blacklist.applyBlacklistTimeout() assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(clock.getTimeMillis(), "1")) assert(blacklist.nextExpiryTime === t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS) // Make sure that we update correctly when we go from having blacklisted executors to @@ -308,6 +333,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M clock.setTime(t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) blacklist.applyBlacklistTimeout() assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(clock.getTimeMillis(), "3")) // we've got one task failure still, but we don't bother setting nextExpiryTime to it, to // avoid wasting time checking for expiry of individual task failures. assert(blacklist.nextExpiryTime === Long.MaxValue) @@ -349,6 +375,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 2)) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) @@ -356,6 +383,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, taskSetBlacklist1.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 2)) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) // Finally, blacklist another executor on the same node as the original blacklisted executor, @@ -365,7 +393,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "3", index = 1) blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2", "3")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "3", 2)) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(0, "hostA", 2)) } test("blacklist still respects legacy configs") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 8c4e389e86a77..0c362b881d912 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -189,6 +189,12 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { assert( !outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter + 3)) } + + test("Duplicate calls to canCommit from the authorized committer gets idempotent responses.") { + val rdd = sc.parallelize(Seq(1), 1) + sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).callCanCommitMultipleTimes _, + 0 until rdd.partitions.size) + } } /** @@ -221,6 +227,16 @@ private case class OutputCommitFunctions(tempDirPath: String) { if (ctx.attemptNumber == 0) failingOutputCommitter else successfulOutputCommitter) } + // Receiver should be idempotent for AskPermissionToCommitOutput + def callCanCommitMultipleTimes(iter: Iterator[Int]): Unit = { + val ctx = TaskContext.get() + val canCommit1 = SparkEnv.get.outputCommitCoordinator + .canCommit(ctx.stageId(), ctx.partitionId(), ctx.attemptNumber()) + val canCommit2 = SparkEnv.get.outputCommitCoordinator + .canCommit(ctx.stageId(), ctx.partitionId(), ctx.attemptNumber()) + assert(canCommit1 && canCommit2) + } + private def runCommitWithProvidedCommitter( ctx: TaskContext, iter: Iterator[Int], diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index e8a88d4909a83..f5575ce1e1577 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -229,7 +229,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match } val numSlices = 16 - val d = sc.parallelize(0 to 1e3.toInt, numSlices).map(w) + val d = sc.parallelize(0 to 10000, numSlices).map(w) d.count() sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) listener.stageInfos.size should be (1) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 2f5b029a966f1..ffb9fe461a486 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -428,7 +428,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock // We don't directly use the application blacklist, but its presence triggers blacklisting // within the taskset. - val blacklistTrackerOpt = Some(new BlacklistTracker(conf, clock)) + val mockListenerBus = mock(classOf[LiveListenerBus]) + val blacklistTrackerOpt = Some(new BlacklistTracker(mockListenerBus, conf, clock)) val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock) { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index f4786e3931c94..4228373036425 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -475,8 +475,8 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val url = new URL( sc.ui.get.webUrl.stripSuffix("/") + "/stages/stage/kill/?id=0") // SPARK-6846: should be POST only but YARN AM doesn't proxy POST - getResponseCode(url, "GET") should be (200) - getResponseCode(url, "POST") should be (200) + TestUtils.httpResponseCode(url, "GET") should be (200) + TestUtils.httpResponseCode(url, "POST") should be (200) } } } @@ -488,8 +488,8 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val url = new URL( sc.ui.get.webUrl.stripSuffix("/") + "/jobs/job/kill/?id=0") // SPARK-6846: should be POST only but YARN AM doesn't proxy POST - getResponseCode(url, "GET") should be (200) - getResponseCode(url, "POST") should be (200) + TestUtils.httpResponseCode(url, "GET") should be (200) + TestUtils.httpResponseCode(url, "POST") should be (200) } } } @@ -671,17 +671,6 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } } - def getResponseCode(url: URL, method: String): Int = { - val connection = url.openConnection().asInstanceOf[HttpURLConnection] - connection.setRequestMethod(method) - try { - connection.connect() - connection.getResponseCode() - } finally { - connection.disconnect() - } - } - def goToUi(sc: SparkContext, path: String): Unit = { goToUi(sc.ui.get, path) } diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 68c7657cb315b..aa67f49185e7f 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -18,12 +18,12 @@ package org.apache.spark.ui import java.net.{BindException, ServerSocket} -import java.net.URI -import javax.servlet.http.HttpServletRequest +import java.net.{URI, URL} +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.io.Source -import org.eclipse.jetty.servlet.ServletContextHandler +import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.mockito.Mockito.{mock, when} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ @@ -167,6 +167,7 @@ class UISuite extends SparkFunSuite { val boundPort = serverInfo.boundPort assert(server.getState === "STARTED") assert(boundPort != 0) + assert(serverInfo.securePort.isDefined) intercept[BindException] { socket = new ServerSocket(boundPort) } @@ -227,8 +228,55 @@ class UISuite extends SparkFunSuite { assert(newHeader === null) } + test("http -> https redirect applies to all URIs") { + var serverInfo: ServerInfo = null + try { + val servlet = new HttpServlet() { + override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = { + res.sendError(HttpServletResponse.SC_OK) + } + } + + def newContext(path: String): ServletContextHandler = { + val ctx = new ServletContextHandler() + ctx.setContextPath(path) + ctx.addServlet(new ServletHolder(servlet), "/root") + ctx + } + + val (conf, sslOptions) = sslEnabledConf() + serverInfo = JettyUtils.startJettyServer("0.0.0.0", 0, sslOptions, + Seq[ServletContextHandler](newContext("/"), newContext("/test1")), + conf) + assert(serverInfo.server.getState === "STARTED") + + val testContext = newContext("/test2") + serverInfo.addHandler(testContext) + testContext.start() + + val httpPort = serverInfo.boundPort + + val tests = Seq( + ("http", serverInfo.boundPort, HttpServletResponse.SC_FOUND), + ("https", serverInfo.securePort.get, HttpServletResponse.SC_OK)) + + tests.foreach { case (scheme, port, expected) => + val urls = Seq( + s"$scheme://localhost:$port/root", + s"$scheme://localhost:$port/test1/root", + s"$scheme://localhost:$port/test2/root") + urls.foreach { url => + val rc = TestUtils.httpResponseCode(new URL(url)) + assert(rc === expected, s"Unexpected status $rc for $url") + } + } + } finally { + stopServer(serverInfo) + } + } + def stopServer(info: ServerInfo): Unit = { - if (info != null && info.server != null) info.server.stop + if (info != null) info.stop() } def closeSocket(socket: ServerSocket): Unit = { diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index da853f1be8b95..e3127da9a6b24 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -408,4 +408,34 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with val newTaskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) assert(newTaskInfo.accumulables === Seq(userAccum)) } + + test("SPARK-19146 drop more elements when stageData.taskData.size > retainedTasks") { + val conf = new SparkConf() + conf.set("spark.ui.retainedTasks", "100") + val taskMetrics = TaskMetrics.empty + taskMetrics.mergeShuffleReadMetrics() + val task = new ShuffleMapTask(0) + val taskType = Utils.getFormattedClassName(task) + + val listener1 = new JobProgressListener(conf) + for (t <- 1 to 101) { + val taskInfo = new TaskInfo(t, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) + taskInfo.finishTime = 1 + listener1.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) + } + // 101 - math.max(100 / 10, 101 - 100) = 91 + assert(listener1.stageIdToData((task.stageId, task.stageAttemptId)).taskData.size === 91) + + val listener2 = new JobProgressListener(conf) + for (t <- 1 to 150) { + val taskInfo = new TaskInfo(t, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) + taskInfo.finishTime = 1 + listener2.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) + } + // 150 - math.max(100 / 10, 150 - 100) = 100 + assert(listener2.stageIdToData((task.stageId, task.stageAttemptId)).taskData.size === 100) + } + } diff --git a/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala index cdd6555697c23..d3a95e399c289 100644 --- a/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala @@ -21,10 +21,6 @@ import org.scalatest.Matchers import org.apache.spark.SparkFunSuite -/** - * - */ - class DistributionSuite extends SparkFunSuite with Matchers { test("summary") { val d = new Distribution((1 to 100).toArray.map{_.toDouble}) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 85da79180fd0b..9f76c74bce89e 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -82,6 +82,12 @@ class JsonProtocolSuite extends SparkFunSuite { val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") + val executorBlacklisted = SparkListenerExecutorBlacklisted(executorBlacklistedTime, "exec1", 22) + val executorUnblacklisted = + SparkListenerExecutorUnblacklisted(executorUnblacklistedTime, "exec1") + val nodeBlacklisted = SparkListenerNodeBlacklisted(nodeBlacklistedTime, "node1", 33) + val nodeUnblacklisted = + SparkListenerNodeUnblacklisted(nodeUnblacklistedTime, "node1") val executorMetricsUpdate = { // Use custom accum ID for determinism val accumUpdates = @@ -109,6 +115,10 @@ class JsonProtocolSuite extends SparkFunSuite { testEvent(applicationEnd, applicationEndJsonString) testEvent(executorAdded, executorAddedJsonString) testEvent(executorRemoved, executorRemovedJsonString) + testEvent(executorBlacklisted, executorBlacklistedJsonString) + testEvent(executorUnblacklisted, executorUnblacklistedJsonString) + testEvent(nodeBlacklisted, nodeBlacklistedJsonString) + testEvent(nodeUnblacklisted, nodeUnblacklistedJsonString) testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString) } @@ -432,6 +442,10 @@ private[spark] object JsonProtocolSuite extends Assertions { private val jobCompletionTime = 1421191296660L private val executorAddedTime = 1421458410000L private val executorRemovedTime = 1421458922000L + private val executorBlacklistedTime = 1421458932000L + private val executorUnblacklistedTime = 1421458942000L + private val nodeBlacklistedTime = 1421458952000L + private val nodeUnblacklistedTime = 1421458962000L private def testEvent(event: SparkListenerEvent, jsonString: String) { val actualJsonString = compact(render(JsonProtocol.sparkEventToJson(event))) @@ -1980,4 +1994,39 @@ private[spark] object JsonProtocolSuite extends Assertions { | ] |} """.stripMargin + + private val executorBlacklistedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorBlacklisted", + | "time" : ${executorBlacklistedTime}, + | "executorId" : "exec1", + | "taskFailures" : 22 + |} + """.stripMargin + private val executorUnblacklistedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted", + | "time" : ${executorUnblacklistedTime}, + | "executorId" : "exec1" + |} + """.stripMargin + private val nodeBlacklistedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeBlacklisted", + | "time" : ${nodeBlacklistedTime}, + | "hostId" : "node1", + | "executorFailures" : 33 + |} + """.stripMargin + private val nodeUnblacklistedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeUnblacklisted", + | "time" : ${nodeUnblacklistedTime}, + | "hostId" : "node1" + |} + """.stripMargin } diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 6be1c72bc6cfb..6d24434ccce7d 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -73,6 +73,8 @@ logs .*dependency-reduced-pom.xml known_translations json_expectation +app-20161115172038-0000 +app-20161116163331-0000 local-1422981759269 local-1422981780767 local-1425081759269 diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations index 0f309908e5996..87bf2f220481d 100644 --- a/dev/create-release/known_translations +++ b/dev/create-release/known_translations @@ -177,7 +177,7 @@ anabranch - Bill Chambers ashangit - Nicolas Fraison avulanov - Alexander Ulanov biglobster - Liang Ke -cenyuhai - Cen Yu Hai +cenyuhai - Yuhai Cen codlife - Jianfei Wang david-weiluo-ren - Weiluo (David) Ren dding3 - Ding Ding @@ -198,7 +198,8 @@ petermaxlee - Peter Lee phalodi - Sandeep Purohit pkch - pkch priyankagargnitk - Priyanka Garg -sharkdtu - Sharkd Tu +sharkdtu - Xiaogang Tu shenh062326 - Shen Hong aokolnychyi - Anton Okolnychyi linbojin - Linbo Jin +lw-lin - Liwei Lin diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index bdae560b2e432..1254188f71680 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -129,13 +129,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.6.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar -parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-column-1.8.2.jar +parquet-common-1.8.2.jar +parquet-encoding-1.8.2.jar +parquet-format-2.3.1.jar +parquet-hadoop-1.8.2.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.8.2.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 2af87c1125bf9..39ba2ae849e68 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -136,13 +136,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.6.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar -parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-column-1.8.2.jar +parquet-common-1.8.2.jar +parquet-encoding-1.8.2.jar +parquet-format-2.3.1.jar +parquet-hadoop-1.8.2.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.8.2.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index fe3656d6ef8f2..d151d12796182 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -136,13 +136,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.6.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar -parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-column-1.8.2.jar +parquet-common-1.8.2.jar +parquet-encoding-1.8.2.jar +parquet-format-2.3.1.jar +parquet-hadoop-1.8.2.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.8.2.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 7b9383d7eeae3..73dc1f9a1398c 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -59,21 +59,21 @@ gson-2.2.4.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar -hadoop-annotations-2.6.4.jar -hadoop-auth-2.6.4.jar -hadoop-client-2.6.4.jar -hadoop-common-2.6.4.jar -hadoop-hdfs-2.6.4.jar -hadoop-mapreduce-client-app-2.6.4.jar -hadoop-mapreduce-client-common-2.6.4.jar -hadoop-mapreduce-client-core-2.6.4.jar -hadoop-mapreduce-client-jobclient-2.6.4.jar -hadoop-mapreduce-client-shuffle-2.6.4.jar -hadoop-yarn-api-2.6.4.jar -hadoop-yarn-client-2.6.4.jar -hadoop-yarn-common-2.6.4.jar -hadoop-yarn-server-common-2.6.4.jar -hadoop-yarn-server-web-proxy-2.6.4.jar +hadoop-annotations-2.6.5.jar +hadoop-auth-2.6.5.jar +hadoop-client-2.6.5.jar +hadoop-common-2.6.5.jar +hadoop-hdfs-2.6.5.jar +hadoop-mapreduce-client-app-2.6.5.jar +hadoop-mapreduce-client-common-2.6.5.jar +hadoop-mapreduce-client-core-2.6.5.jar +hadoop-mapreduce-client-jobclient-2.6.5.jar +hadoop-mapreduce-client-shuffle-2.6.5.jar +hadoop-yarn-api-2.6.5.jar +hadoop-yarn-client-2.6.5.jar +hadoop-yarn-common-2.6.5.jar +hadoop-yarn-server-common-2.6.5.jar +hadoop-yarn-server-web-proxy-2.6.5.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar @@ -144,13 +144,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.6.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar -parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-column-1.8.2.jar +parquet-common-1.8.2.jar +parquet-encoding-1.8.2.jar +parquet-format-2.3.1.jar +parquet-hadoop-1.8.2.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.8.2.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 2e486fdc00f77..6bf0923a1d751 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -145,13 +145,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.6.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar -parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-column-1.8.2.jar +parquet-common-1.8.2.jar +parquet-encoding-1.8.2.jar +parquet-format-2.3.1.jar +parquet-hadoop-1.8.2.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.8.2.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 47ff504316d38..6fb25f3b98722 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -220,6 +220,8 @@ cp -r "$SPARK_HOME/data" "$DISTDIR" if [ "$MAKE_PIP" == "true" ]; then echo "Building python distribution package" pushd "$SPARK_HOME/python" > /dev/null + # Delete the egg info file if it exists, this can cache older setup files. + rm -rf pyspark.egg-info || echo "No existing egg info file, skipping deletion" python setup.py sdist popd > /dev/null else diff --git a/dev/pip-sanity-check.py b/dev/pip-sanity-check.py index 430c2ab52766a..c491005f49719 100644 --- a/dev/pip-sanity-check.py +++ b/dev/pip-sanity-check.py @@ -18,6 +18,8 @@ from __future__ import print_function from pyspark.sql import SparkSession +from pyspark.ml.param import Params +from pyspark.mllib.linalg import * import sys if __name__ == "__main__": diff --git a/dev/requirements.txt b/dev/requirements.txt index bf042d22a8b47..79782279f8fbd 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -1,3 +1,4 @@ jira==1.0.3 PyGithub==1.26.0 Unidecode==0.04.19 +pypandoc==1.3.3 diff --git a/dev/run-pip-tests b/dev/run-pip-tests index e1da18e60bb3d..af1b1feb70cd1 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -78,11 +78,14 @@ for python in "${PYTHON_EXECS[@]}"; do mkdir -p "$VIRTUALENV_PATH" virtualenv --python=$python "$VIRTUALENV_PATH" source "$VIRTUALENV_PATH"/bin/activate - # Upgrade pip - pip install --upgrade pip + # Upgrade pip & friends + pip install --upgrade pip pypandoc wheel + pip install numpy # Needed so we can verify mllib imports echo "Creating pip installable source dist" cd "$FWDIR"/python + # Delete the egg info file if it exists, this can cache the setup file. + rm -rf pyspark.egg-info || echo "No existing egg info file, skipping deletion" $python setup.py sdist diff --git a/docs/configuration.md b/docs/configuration.md index 7a11a983d5972..7c040330db637 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -435,7 +435,7 @@ Apart from these, the following properties are also available, and may be useful spark.jars.packages - Comma-separated list of maven coordinates of jars to include on the driver and executor + Comma-separated list of Maven coordinates of jars to include on the driver and executor classpaths. The coordinates should be groupId:artifactId:version. If spark.jars.ivySettings is given artifacts will be resolved according to the configuration in the file, otherwise artifacts will be searched for in the local maven repo, then maven central and finally any additional remote @@ -684,24 +684,24 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedJobs 1000 - How many jobs the Spark UI and status APIs remember before garbage - collecting. + How many jobs the Spark UI and status APIs remember before garbage collecting. + This is a target maximum, and fewer elements may be retained in some circumstances. spark.ui.retainedStages 1000 - How many stages the Spark UI and status APIs remember before garbage - collecting. + How many stages the Spark UI and status APIs remember before garbage collecting. + This is a target maximum, and fewer elements may be retained in some circumstances. spark.ui.retainedTasks 100000 - How many tasks the Spark UI and status APIs remember before garbage - collecting. + How many tasks the Spark UI and status APIs remember before garbage collecting. + This is a target maximum, and fewer elements may be retained in some circumstances. @@ -1639,40 +1639,40 @@ Apart from these, the following properties are also available, and may be useful - spark.authenticate.enableSaslEncryption + spark.network.crypto.enabled false - Enable encrypted communication when authentication is - enabled. This is supported by the block transfer service and the - RPC endpoints. + Enable encryption using the commons-crypto library for RPC and block transfer service. + Requires spark.authenticate to be enabled. - spark.network.sasl.serverAlwaysEncrypt - false + spark.network.crypto.keyLength + 128 - Disable unencrypted connections for services that support SASL authentication. This is - currently supported by the external shuffle service. + The length in bits of the encryption key to generate. Valid values are 128, 192 and 256. - spark.network.aes.enabled - false + spark.network.crypto.keyFactoryAlgorithm + PBKDF2WithHmacSHA1 - Enable AES for over-the-wire encryption. This is supported for RPC and the block transfer service. - This option has precedence over SASL-based encryption if both are enabled. + The key factory algorithm to use when generating encryption keys. Should be one of the + algorithms supported by the javax.crypto.SecretKeyFactory class in the JRE being used. - spark.network.aes.keySize - 16 + spark.network.crypto.saslFallback + true - The bytes of AES cipher key which is effective when AES cipher is enabled. AES - works with 16, 24 and 32 bytes keys. + Whether to fall back to SASL authentication if authentication fails using Spark's internal + mechanism. This is useful when the application is connecting to old shuffle services that + do not support the internal Spark authentication protocol. On the server side, this can be + used to block older clients from authenticating against a new shuffle service. - spark.network.aes.config.* + spark.network.crypto.config.* None Configuration values for the commons-crypto library, such as which cipher implementations to @@ -1680,6 +1680,22 @@ Apart from these, the following properties are also available, and may be useful "commons.crypto" prefix. + + spark.authenticate.enableSaslEncryption + false + + Enable encrypted communication when authentication is + enabled. This is supported by the block transfer service and the + RPC endpoints. + + + + spark.network.sasl.serverAlwaysEncrypt + false + + Disable unencrypted connections for services that support SASL authentication. + + spark.core.connection.ack.wait.timeout spark.network.timeout diff --git a/docs/index.md b/docs/index.md index 57b9fa848f4a3..023e06ada3690 100644 --- a/docs/index.md +++ b/docs/index.md @@ -15,7 +15,7 @@ It also supports a rich set of higher-level tools including [Spark SQL](sql-prog Get Spark from the [downloads page](http://spark.apache.org/downloads.html) of the project website. This documentation is for Spark version {{site.SPARK_VERSION}}. Spark uses Hadoop's client libraries for HDFS and YARN. Downloads are pre-packaged for a handful of popular Hadoop versions. Users can also download a "Hadoop free" binary and run Spark with any Hadoop version [by augmenting Spark's classpath](hadoop-provided.html). -Scala and Java users can include Spark in their projects using its maven cooridnates and in the future Python users can also install Spark from PyPI. +Scala and Java users can include Spark in their projects using its Maven coordinates and in the future Python users can also install Spark from PyPI. If you'd like to build Spark from diff --git a/docs/monitoring.md b/docs/monitoring.md index e918174e2b57c..7ba4824d463fc 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -267,12 +267,23 @@ can be identified by their `[attempt-id]`. In the API listed below, when running
?status=[completed|running] list only applications in the chosen state.
- ?minDate=[date] earliest date/time to list. + ?minDate=[date] earliest start date/time to list. +
+ ?maxDate=[date] latest start date/time to list. +
+ ?minEndDate=[date] earliest end date/time to list. +
+ ?maxEndDate=[date] latest end date/time to list. +
+ ?limit=[limit] limits the number of applications listed.
Examples:
?minDate=2015-02-10
?minDate=2015-02-03T16:42:40.000GMT -
?maxDate=[date] latest date/time to list; uses same format as minDate. -
?limit=[limit] limits the number of applications listed. +
?maxDate=2015-02-11T20:41:30.000GMT +
?minEndDate=2015-02-12 +
?minEndDate=2015-02-12T09:15:10.000GMT +
?maxEndDate=2015-02-14T16:30:45.000GMT +
?limit=10 /applications/[app-id]/jobs diff --git a/docs/programming-guide.md b/docs/programming-guide.md index a4017b5b97550..db8b048fcef94 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -185,7 +185,7 @@ In the Spark shell, a special interpreter-aware SparkContext is already created variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `--master` argument, and you can add JARs to the classpath by passing a comma-separated list to the `--jars` argument. You can also add dependencies -(e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates +(e.g. Spark Packages) to your shell session by supplying a comma-separated list of Maven coordinates to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. Sonatype) can be passed to the `--repositories` argument. For example, to run `bin/spark-shell` on exactly four cores, use: @@ -200,7 +200,7 @@ Or, to also add `code.jar` to its classpath, use: $ ./bin/spark-shell --master local[4] --jars code.jar {% endhighlight %} -To include a dependency using maven coordinates: +To include a dependency using Maven coordinates: {% highlight bash %} $ ./bin/spark-shell --master local[4] --packages "org.example:example:0.1" @@ -217,7 +217,7 @@ In the PySpark shell, a special interpreter-aware SparkContext is already create variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `--master` argument, and you can add Python .zip, .egg or .py files to the runtime path by passing a comma-separated list to `--py-files`. You can also add dependencies -(e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates +(e.g. Spark Packages) to your shell session by supplying a comma-separated list of Maven coordinates to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. Sonatype) can be passed to the `--repositories` argument. Any Python dependencies a Spark package has (listed in the requirements.txt of that package) must be manually installed using `pip` when necessary. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index f7513454c7852..051f64e1beabf 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -276,15 +276,16 @@ To use a custom metrics.properties for the application master and executors, upd - spark.yarn.access.namenodes + spark.yarn.access.hadoopFileSystems (none) - A comma-separated list of secure HDFS namenodes your Spark application is going to access. For - example, spark.yarn.access.namenodes=hdfs://nn1.com:8032,hdfs://nn2.com:8032, - webhdfs://nn3.com:50070. The Spark application must have access to the namenodes listed + A comma-separated list of secure Hadoop filesystems your Spark application is going to access. For + example, spark.yarn.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032, + webhdfs://nn3.com:50070. The Spark application must have access to the filesystems listed and Kerberos must be properly configured to be able to access them (either in the same realm - or in a trusted realm). Spark acquires security tokens for each of the namenodes so that - the Spark application can access those remote HDFS clusters. + or in a trusted realm). Spark acquires security tokens for each of the filesystems so that + the Spark application can access those remote Hadoop filesystems. spark.yarn.access.namenodes + is deprecated, please use this instead. @@ -496,10 +497,10 @@ includes a URI of the metadata store in `"hive.metastore.uris`, and If an application needs to interact with other secure Hadoop filesystems, then the tokens needed to access these clusters must be explicitly requested at -launch time. This is done by listing them in the `spark.yarn.access.namenodes` property. +launch time. This is done by listing them in the `spark.yarn.access.hadoopFileSystems` property. ``` -spark.yarn.access.namenodes hdfs://ireland.example.org:8020/,webhdfs://frankfurt.example.org:50070/ +spark.yarn.access.hadoopFileSystems hdfs://ireland.example.org:8020/,webhdfs://frankfurt.example.org:50070/ ``` Spark supports integrating with other security-aware services through Java Services mechanism (see @@ -574,7 +575,7 @@ spark.yarn.security.credentials.hive.enabled false spark.yarn.security.credentials.hbase.enabled false ``` -The configuration option `spark.yarn.access.namenodes` must be unset. +The configuration option `spark.yarn.access.hadoopFileSystems` must be unset. ## Troubleshooting Kerberos diff --git a/docs/security.md b/docs/security.md index baadfefbec826..67956930fe5d9 100644 --- a/docs/security.md +++ b/docs/security.md @@ -82,6 +82,7 @@ Key-stores can be generated by `keytool` program. The reference documentation fo [here](https://docs.oracle.com/javase/7/docs/technotes/tools/solaris/keytool.html). The most basic steps to configure the key-stores and the trust-store for the standalone deployment mode is as follows: + * Generate a keys pair for each node * Export the public key of the key pair to a file on each node * Import all exported public keys into a single trust-store diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index f4c89e58fa431..9cf480caba3e4 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -382,6 +382,52 @@ For example: +## Aggregations + +The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common +aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, etc. +While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in +[Scala](api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$) and +[Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work with strongly typed Datasets. +Moreover, users are not limited to the predefined aggregate functions and can create their own. + +### Untyped User-Defined Aggregate Functions + +
+ +
+ +Users have to extend the [UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction) +abstract class to implement a custom untyped aggregate function. For example, a user-defined average +can look like: + +{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%} +
+ +
+ +{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%} +
+ +
+ +### Type-Safe User-Defined Aggregate Functions + +User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. +For example, a type-safe user-defined average can look like: +
+ +
+ +{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%} +
+ +
+ +{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%} +
+ +
# Data Sources @@ -1089,7 +1135,7 @@ Tables from the remote database can be loaded as a DataFrame or Spark SQL tempor the Data Sources API. Users can specify the JDBC connection properties in the data source options. user and password are normally provided as connection properties for logging into the data sources. In addition to the connection properties, Spark also supports -the following case-sensitive options: +the following case-insensitive options: diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index b645d3c3a4b53..6ef54ac210704 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -183,7 +183,7 @@ stream.foreachRDD(new VoidFunction>>() { Note that the typecast to `HasOffsetRanges` will only succeed if it is done in the first method called on the result of `createDirectStream`, not later down a chain of methods. Be aware that the one-to-one mapping between RDD partition and Kafka partition does not remain after any methods that shuffle or repartition, e.g. reduceByKey() or window(). ### Storing Offsets -Kafka delivery semantics in the case of failure depend on how and when offsets are stored. Spark output operations are [at-least-once](streaming-programming-guide.html#semantics-of-output-operations). So if you want the equivalent of exactly-once semantics, you must either store offsets after an idempotent output, or store offsets in an atomic transaction alongside output. With this integration, you have 3 options, in order of increasing reliablity (and code complexity), for how to store offsets. +Kafka delivery semantics in the case of failure depend on how and when offsets are stored. Spark output operations are [at-least-once](streaming-programming-guide.html#semantics-of-output-operations). So if you want the equivalent of exactly-once semantics, you must either store offsets after an idempotent output, or store offsets in an atomic transaction alongside output. With this integration, you have 3 options, in order of increasing reliability (and code complexity), for how to store offsets. #### Checkpoints If you enable Spark [checkpointing](streaming-programming-guide.html#checkpointing), offsets will be stored in the checkpoint. This is easy to enable, but there are drawbacks. Your output operation must be idempotent, since you will get repeated outputs; transactions are not an option. Furthermore, you cannot recover from a checkpoint if your application code has changed. For planned upgrades, you can mitigate this by running the new code at the same time as the old code (since outputs need to be idempotent anyway, they should not clash). But for unplanned failures that require code changes, you will lose data unless you have another way to identify known good starting offsets. diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index b8b4cc3a53046..d23dbcf10d952 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -189,7 +189,7 @@ This can use up a significant amount of space over time and will need to be clea is handled automatically, and with Spark standalone, automatic cleanup can be configured with the `spark.worker.cleanup.appDataTtl` property. -Users may also include any other dependencies by supplying a comma-delimited list of maven coordinates +Users may also include any other dependencies by supplying a comma-delimited list of Maven coordinates with `--packages`. All transitive dependencies will be handled when using this command. Additional repositories (or resolvers in SBT) can be added in a comma-delimited fashion with the flag `--repositories`. (Note that credentials for password-protected repositories can be supplied in some cases in the repository URI, diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java new file mode 100644 index 0000000000000..78e9011be4705 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql; + +// $example on:typed_custom_aggregation$ +import java.io.Serializable; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.TypedColumn; +import org.apache.spark.sql.expressions.Aggregator; +// $example off:typed_custom_aggregation$ + +public class JavaUserDefinedTypedAggregation { + + // $example on:typed_custom_aggregation$ + public static class Employee implements Serializable { + private String name; + private long salary; + + // Constructors, getters, setters... + // $example off:typed_custom_aggregation$ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public long getSalary() { + return salary; + } + + public void setSalary(long salary) { + this.salary = salary; + } + // $example on:typed_custom_aggregation$ + } + + public static class Average implements Serializable { + private long sum; + private long count; + + // Constructors, getters, setters... + // $example off:typed_custom_aggregation$ + public Average() { + } + + public Average(long sum, long count) { + this.sum = sum; + this.count = count; + } + + public long getSum() { + return sum; + } + + public void setSum(long sum) { + this.sum = sum; + } + + public long getCount() { + return count; + } + + public void setCount(long count) { + this.count = count; + } + // $example on:typed_custom_aggregation$ + } + + public static class MyAverage extends Aggregator { + // A zero value for this aggregation. Should satisfy the property that any b + zero = b + public Average zero() { + return new Average(0L, 0L); + } + // Combine two values to produce a new value. For performance, the function may modify `buffer` + // and return it instead of constructing a new object + public Average reduce(Average buffer, Employee employee) { + long newSum = buffer.getSum() + employee.getSalary(); + long newCount = buffer.getCount() + 1; + buffer.setSum(newSum); + buffer.setCount(newCount); + return buffer; + } + // Merge two intermediate values + public Average merge(Average b1, Average b2) { + long mergedSum = b1.getSum() + b2.getSum(); + long mergedCount = b1.getCount() + b2.getCount(); + b1.setSum(mergedSum); + b1.setCount(mergedCount); + return b1; + } + // Transform the output of the reduction + public Double finish(Average reduction) { + return ((double) reduction.getSum()) / reduction.getCount(); + } + // Specifies the Encoder for the intermediate value type + public Encoder bufferEncoder() { + return Encoders.bean(Average.class); + } + // Specifies the Encoder for the final output value type + public Encoder outputEncoder() { + return Encoders.DOUBLE(); + } + } + // $example off:typed_custom_aggregation$ + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL user-defined Datasets aggregation example") + .getOrCreate(); + + // $example on:typed_custom_aggregation$ + Encoder employeeEncoder = Encoders.bean(Employee.class); + String path = "examples/src/main/resources/employees.json"; + Dataset ds = spark.read().json(path).as(employeeEncoder); + ds.show(); + // +-------+------+ + // | name|salary| + // +-------+------+ + // |Michael| 3000| + // | Andy| 4500| + // | Justin| 3500| + // | Berta| 4000| + // +-------+------+ + + MyAverage myAverage = new MyAverage(); + // Convert the function to a `TypedColumn` and give it a name + TypedColumn averageSalary = myAverage.toColumn().name("average_salary"); + Dataset result = ds.select(averageSalary); + result.show(); + // +--------------+ + // |average_salary| + // +--------------+ + // | 3750.0| + // +--------------+ + // $example off:typed_custom_aggregation$ + spark.stop(); + } + +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java new file mode 100644 index 0000000000000..6da60a1fc6b88 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql; + +// $example on:untyped_custom_aggregation$ +import java.util.ArrayList; +import java.util.List; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.expressions.MutableAggregationBuffer; +import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off:untyped_custom_aggregation$ + +public class JavaUserDefinedUntypedAggregation { + + // $example on:untyped_custom_aggregation$ + public static class MyAverage extends UserDefinedAggregateFunction { + + private StructType inputSchema; + private StructType bufferSchema; + + public MyAverage() { + List inputFields = new ArrayList<>(); + inputFields.add(DataTypes.createStructField("inputColumn", DataTypes.LongType, true)); + inputSchema = DataTypes.createStructType(inputFields); + + List bufferFields = new ArrayList<>(); + bufferFields.add(DataTypes.createStructField("sum", DataTypes.LongType, true)); + bufferFields.add(DataTypes.createStructField("count", DataTypes.LongType, true)); + bufferSchema = DataTypes.createStructType(bufferFields); + } + // Data types of input arguments of this aggregate function + public StructType inputSchema() { + return inputSchema; + } + // Data types of values in the aggregation buffer + public StructType bufferSchema() { + return bufferSchema; + } + // The data type of the returned value + public DataType dataType() { + return DataTypes.DoubleType; + } + // Whether this function always returns the same output on the identical input + public boolean deterministic() { + return true; + } + // Initializes the given aggregation buffer. The buffer itself is a `Row` that in addition to + // standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides + // the opportunity to update its values. Note that arrays and maps inside the buffer are still + // immutable. + public void initialize(MutableAggregationBuffer buffer) { + buffer.update(0, 0L); + buffer.update(1, 0L); + } + // Updates the given aggregation buffer `buffer` with new input data from `input` + public void update(MutableAggregationBuffer buffer, Row input) { + if (!input.isNullAt(0)) { + long updatedSum = buffer.getLong(0) + input.getLong(0); + long updatedCount = buffer.getLong(1) + 1; + buffer.update(0, updatedSum); + buffer.update(1, updatedCount); + } + } + // Merges two aggregation buffers and stores the updated buffer values back to `buffer1` + public void merge(MutableAggregationBuffer buffer1, Row buffer2) { + long mergedSum = buffer1.getLong(0) + buffer2.getLong(0); + long mergedCount = buffer1.getLong(1) + buffer2.getLong(1); + buffer1.update(0, mergedSum); + buffer1.update(1, mergedCount); + } + // Calculates the final result + public Double evaluate(Row buffer) { + return ((double) buffer.getLong(0)) / buffer.getLong(1); + } + } + // $example off:untyped_custom_aggregation$ + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL user-defined DataFrames aggregation example") + .getOrCreate(); + + // $example on:untyped_custom_aggregation$ + // Register the function to access it + spark.udf().register("myAverage", new MyAverage()); + + Dataset df = spark.read().json("examples/src/main/resources/employees.json"); + df.createOrReplaceTempView("employees"); + df.show(); + // +-------+------+ + // | name|salary| + // +-------+------+ + // |Michael| 3000| + // | Andy| 4500| + // | Justin| 3500| + // | Berta| 4000| + // +-------+------+ + + Dataset result = spark.sql("SELECT myAverage(salary) as average_salary FROM employees"); + result.show(); + // +--------------+ + // |average_salary| + // +--------------+ + // | 3750.0| + // +--------------+ + // $example off:untyped_custom_aggregation$ + + spark.stop(); + } +} diff --git a/examples/src/main/resources/employees.json b/examples/src/main/resources/employees.json new file mode 100644 index 0000000000000..6b2e6329a1cb2 --- /dev/null +++ b/examples/src/main/resources/employees.json @@ -0,0 +1,4 @@ +{"name":"Michael", "salary":3000} +{"name":"Andy", "salary":4500} +{"name":"Justin", "salary":3500} +{"name":"Berta", "salary":4000} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala index a4f62e78710d4..c2852aacb05d5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala @@ -21,7 +21,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.Binarizer // $example off$ -import org.apache.spark.sql.{SparkSession} +import org.apache.spark.sql.SparkSession object BinarizerExample { def main(args: Array[String]): Unit = { diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala index f27c403c5b388..b9a612d96a577 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala @@ -16,10 +16,6 @@ */ package org.apache.spark.examples.sql -// $example on:schema_inferring$ -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.Encoder -// $example off:schema_inferring$ import org.apache.spark.sql.Row // $example on:init_session$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala new file mode 100644 index 0000000000000..ac617d19d36cf --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql + +// $example on:typed_custom_aggregation$ +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.Encoders +import org.apache.spark.sql.SparkSession +// $example off:typed_custom_aggregation$ + +object UserDefinedTypedAggregation { + + // $example on:typed_custom_aggregation$ + case class Employee(name: String, salary: Long) + case class Average(var sum: Long, var count: Long) + + object MyAverage extends Aggregator[Employee, Average, Double] { + // A zero value for this aggregation. Should satisfy the property that any b + zero = b + def zero: Average = Average(0L, 0L) + // Combine two values to produce a new value. For performance, the function may modify `buffer` + // and return it instead of constructing a new object + def reduce(buffer: Average, employee: Employee): Average = { + buffer.sum += employee.salary + buffer.count += 1 + buffer + } + // Merge two intermediate values + def merge(b1: Average, b2: Average): Average = { + b1.sum += b2.sum + b1.count += b2.count + b1 + } + // Transform the output of the reduction + def finish(reduction: Average): Double = reduction.sum.toDouble / reduction.count + // Specifies the Encoder for the intermediate value type + def bufferEncoder: Encoder[Average] = Encoders.product + // Specifies the Encoder for the final output value type + def outputEncoder: Encoder[Double] = Encoders.scalaDouble + } + // $example off:typed_custom_aggregation$ + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder() + .appName("Spark SQL user-defined Datasets aggregation example") + .getOrCreate() + + import spark.implicits._ + + // $example on:typed_custom_aggregation$ + val ds = spark.read.json("examples/src/main/resources/employees.json").as[Employee] + ds.show() + // +-------+------+ + // | name|salary| + // +-------+------+ + // |Michael| 3000| + // | Andy| 4500| + // | Justin| 3500| + // | Berta| 4000| + // +-------+------+ + + // Convert the function to a `TypedColumn` and give it a name + val averageSalary = MyAverage.toColumn.name("average_salary") + val result = ds.select(averageSalary) + result.show() + // +--------------+ + // |average_salary| + // +--------------+ + // | 3750.0| + // +--------------+ + // $example off:typed_custom_aggregation$ + + spark.stop() + } + +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala new file mode 100644 index 0000000000000..9c9ebc55163de --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql + +// $example on:untyped_custom_aggregation$ +import org.apache.spark.sql.expressions.MutableAggregationBuffer +import org.apache.spark.sql.expressions.UserDefinedAggregateFunction +import org.apache.spark.sql.types._ +import org.apache.spark.sql.Row +import org.apache.spark.sql.SparkSession +// $example off:untyped_custom_aggregation$ + +object UserDefinedUntypedAggregation { + + // $example on:untyped_custom_aggregation$ + object MyAverage extends UserDefinedAggregateFunction { + // Data types of input arguments of this aggregate function + def inputSchema: StructType = StructType(StructField("inputColumn", LongType) :: Nil) + // Data types of values in the aggregation buffer + def bufferSchema: StructType = { + StructType(StructField("sum", LongType) :: StructField("count", LongType) :: Nil) + } + // The data type of the returned value + def dataType: DataType = DoubleType + // Whether this function always returns the same output on the identical input + def deterministic: Boolean = true + // Initializes the given aggregation buffer. The buffer itself is a `Row` that in addition to + // standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides + // the opportunity to update its values. Note that arrays and maps inside the buffer are still + // immutable. + def initialize(buffer: MutableAggregationBuffer): Unit = { + buffer(0) = 0L + buffer(1) = 0L + } + // Updates the given aggregation buffer `buffer` with new input data from `input` + def update(buffer: MutableAggregationBuffer, input: Row): Unit = { + if (!input.isNullAt(0)) { + buffer(0) = buffer.getLong(0) + input.getLong(0) + buffer(1) = buffer.getLong(1) + 1 + } + } + // Merges two aggregation buffers and stores the updated buffer values back to `buffer1` + def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit = { + buffer1(0) = buffer1.getLong(0) + buffer2.getLong(0) + buffer1(1) = buffer1.getLong(1) + buffer2.getLong(1) + } + // Calculates the final result + def evaluate(buffer: Row): Double = buffer.getLong(0).toDouble / buffer.getLong(1) + } + // $example off:untyped_custom_aggregation$ + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder() + .appName("Spark SQL user-defined DataFrames aggregation example") + .getOrCreate() + + // $example on:untyped_custom_aggregation$ + // Register the function to access it + spark.udf.register("myAverage", MyAverage) + + val df = spark.read.json("examples/src/main/resources/employees.json") + df.createOrReplaceTempView("employees") + df.show() + // +-------+------+ + // | name|salary| + // +-------+------+ + // |Michael| 3000| + // | Andy| 4500| + // | Justin| 3500| + // | Berta| 4000| + // +-------+------+ + + val result = spark.sql("SELECT myAverage(salary) as average_salary FROM employees") + result.show() + // +--------------+ + // |average_salary| + // +--------------+ + // | 3750.0| + // +--------------+ + // $example off:untyped_custom_aggregation$ + + spark.stop() + } + +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala index f0756c4e183c9..de477c5ce8161 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.sql.streaming -import org.apache.spark.sql.functions._ import org.apache.spark.sql.SparkSession /** diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index c9325dea0bb04..a1a065a443e67 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -51,12 +51,17 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { + "B'1000100101', E'\\\\xDEADBEEF', true, '172.16.0.42', '192.168.0.0/16', " + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}', '{0.11, 0.22}', 'd1', 1.01, 1)""" ).executeUpdate() + conn.prepareStatement("INSERT INTO bar VALUES (null, null, null, null, null, " + + "null, null, null, null, null, " + + "null, null, null, null, null, null, null)" + ).executeUpdate() } test("Type mapping for various types") { val df = sqlContext.read.jdbc(jdbcUrl, "bar", new Properties) - val rows = df.collect() - assert(rows.length == 1) + val rows = df.collect().sortBy(_.toString()) + assert(rows.length == 2) + // Test the types, and values using the first row. val types = rows(0).toSeq.map(x => x.getClass) assert(types.length == 17) assert(classOf[String].isAssignableFrom(types(0))) @@ -96,6 +101,9 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows(0).getString(14) == "d1") assert(rows(0).getFloat(15) == 1.01f) assert(rows(0).getShort(16) == 1) + + // Test reading null values using the second row. + assert(0.until(16).forall(rows(1).isNullAt(_))) } test("Basic write test") { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 43b8d9d6d7eef..8e6c93e672230 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -24,7 +24,7 @@ import java.nio.charset.StandardCharsets import scala.collection.JavaConverters._ import scala.util.control.NonFatal -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer, OffsetOutOfRangeException} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.common.TopicPartition diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala index 8f38095208117..bf8adbe42f3f5 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } import org.apache.kafka.common.TopicPartition -import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} +import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala index 37046329e5617..e6bdef04512d5 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala @@ -25,7 +25,6 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } -import org.apache.spark.api.java.function.{ Function0 => JFunction0 } import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.streaming.StreamingContext diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index 859fe9edb44fc..a70c13d7d68a8 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.util.Random -import com.amazonaws.auth.{BasicAWSCredentials, DefaultAWSCredentialsProviderChain} +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain import com.amazonaws.regions.RegionUtils import com.amazonaws.services.kinesis.AmazonKinesisClient import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala index 70b5cc7ca0e8e..c445c15a5f644 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala @@ -26,7 +26,7 @@ import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason import org.apache.spark.internal.Logging import org.apache.spark.streaming.Duration import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Clock, SystemClock, ThreadUtils} +import org.apache.spark.util.{Clock, SystemClock} /** * This is a helper class for managing Kinesis checkpointing. @@ -64,7 +64,20 @@ private[kinesis] class KinesisCheckpointer( def removeCheckpointer(shardId: String, checkpointer: IRecordProcessorCheckpointer): Unit = { synchronized { checkpointers.remove(shardId) - checkpoint(shardId, checkpointer) + } + if (checkpointer != null) { + try { + // We must call `checkpoint()` with no parameter to finish reading shards. + // See an URL below for details: + // https://forums.aws.amazon.com/thread.jspa?threadID=244218 + KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(), 4, 100) + } catch { + case NonFatal(e) => + logError(s"Exception: WorkerId $workerId encountered an exception while checkpointing" + + s"to finish reading a shard of $shardId.", e) + // Rethrow the exception to the Kinesis Worker that is managing this RecordProcessor + throw e + } } } diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala index 0fe66254e989d..f183ef00b33cd 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -40,11 +40,10 @@ import org.apache.spark.internal.Logging * * PLEASE KEEP THIS FILE UNDER src/main AS PYTHON TESTS NEED ACCESS TO THIS FILE! */ -private[kinesis] class KinesisTestUtils extends Logging { +private[kinesis] class KinesisTestUtils(streamShardCount: Int = 2) extends Logging { val endpointUrl = KinesisTestUtils.endpointUrl val regionName = RegionUtils.getRegionByEndpoint(endpointUrl).getName() - val streamShardCount = 2 private val createStreamTimeoutSeconds = 300 private val describeStreamPollTimeSeconds = 1 @@ -88,7 +87,7 @@ private[kinesis] class KinesisTestUtils extends Logging { logInfo(s"Creating stream ${_streamName}") val createStreamRequest = new CreateStreamRequest() createStreamRequest.setStreamName(_streamName) - createStreamRequest.setShardCount(2) + createStreamRequest.setShardCount(streamShardCount) kinesisClient.createStream(createStreamRequest) // The stream is now being created. Wait for it to become active. @@ -97,6 +96,31 @@ private[kinesis] class KinesisTestUtils extends Logging { logInfo(s"Created stream ${_streamName}") } + def getShards(): Seq[Shard] = { + kinesisClient.describeStream(_streamName).getStreamDescription.getShards.asScala + } + + def splitShard(shardId: String): Unit = { + val splitShardRequest = new SplitShardRequest() + splitShardRequest.withStreamName(_streamName) + splitShardRequest.withShardToSplit(shardId) + // Set a half of the max hash value + splitShardRequest.withNewStartingHashKey("170141183460469231731687303715884105728") + kinesisClient.splitShard(splitShardRequest) + // Wait for the shards to become active + waitForStreamToBeActive(_streamName) + } + + def mergeShard(shardToMerge: String, adjacentShardToMerge: String): Unit = { + val mergeShardRequest = new MergeShardsRequest + mergeShardRequest.withStreamName(_streamName) + mergeShardRequest.withShardToMerge(shardToMerge) + mergeShardRequest.withAdjacentShardToMerge(adjacentShardToMerge) + kinesisClient.mergeShards(mergeShardRequest) + // Wait for the shards to become active + waitForStreamToBeActive(_streamName) + } + /** * Push data to Kinesis stream and return a map of * shardId -> seq of (data, seq number) pushed to corresponding shard diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala index 0b455e574e6fa..2ee3224b3c286 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala @@ -25,7 +25,8 @@ import scala.collection.mutable.ArrayBuffer import com.amazonaws.services.kinesis.producer.{KinesisProducer => KPLProducer, KinesisProducerConfiguration, UserRecordResult} import com.google.common.util.concurrent.{FutureCallback, Futures} -private[kinesis] class KPLBasedKinesisTestUtils extends KinesisTestUtils { +private[kinesis] class KPLBasedKinesisTestUtils(streamShardCount: Int = 2) + extends KinesisTestUtils(streamShardCount) { override protected def getProducer(aggregate: Boolean): KinesisDataGenerator = { if (!aggregate) { new SimpleDataGenerator(kinesisClient) diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala index e1499a8220991..fef24ed4c5dd0 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.kinesis -import java.util.concurrent.{ExecutorService, TimeoutException} +import java.util.concurrent.TimeoutException import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ @@ -30,7 +30,6 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.Eventually -import org.scalatest.concurrent.Eventually._ import org.scalatest.mock.MockitoSugar import org.apache.spark.streaming.{Duration, TestSuiteBase} @@ -119,7 +118,7 @@ class KinesisCheckpointerSuite extends TestSuiteBase when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) kinesisCheckpointer.removeCheckpointer(shardId, checkpointerMock) - verify(checkpointerMock, times(1)).checkpoint(anyString()) + verify(checkpointerMock, times(1)).checkpoint() } test("if checkpointing is going on, wait until finished before removing and checkpointing") { @@ -146,7 +145,8 @@ class KinesisCheckpointerSuite extends TestSuiteBase clock.advance(checkpointInterval.milliseconds / 2) eventually(timeout(1 second)) { - verify(checkpointerMock, times(2)).checkpoint(anyString()) + verify(checkpointerMock, times(1)).checkpoint(anyString) + verify(checkpointerMock, times(1)).checkpoint() } } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index 0e71bf9b84332..404b673c01171 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -225,6 +225,76 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun ssc.stop(stopSparkContext = false) } + testIfEnabled("split and merge shards in a stream") { + // Since this test tries to split and merge shards in a stream, we create another + // temporary stream and then remove it when finished. + val localAppName = s"KinesisStreamSuite-${math.abs(Random.nextLong())}" + val localTestUtils = new KPLBasedKinesisTestUtils(1) + localTestUtils.createStream() + try { + val awsCredentials = KinesisTestUtils.getAWSCredentials() + val stream = KinesisUtils.createStream(ssc, localAppName, localTestUtils.streamName, + localTestUtils.endpointUrl, localTestUtils.regionName, InitialPositionInStream.LATEST, + Seconds(10), StorageLevel.MEMORY_ONLY, + awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey) + + val collected = new mutable.HashSet[Int] + stream.map { bytes => new String(bytes).toInt }.foreachRDD { rdd => + collected.synchronized { + collected ++= rdd.collect() + logInfo("Collected = " + collected.mkString(", ")) + } + } + ssc.start() + + val testData1 = 1 to 10 + val testData2 = 11 to 20 + val testData3 = 21 to 30 + + eventually(timeout(60 seconds), interval(10 second)) { + localTestUtils.pushData(testData1, aggregateTestData) + assert(collected.synchronized { collected === testData1.toSet }, + "\nData received does not match data sent") + } + + val shardToSplit = localTestUtils.getShards().head + localTestUtils.splitShard(shardToSplit.getShardId) + val (splitOpenShards, splitCloseShards) = localTestUtils.getShards().partition { shard => + shard.getSequenceNumberRange.getEndingSequenceNumber == null + } + + // We should have one closed shard and two open shards + assert(splitCloseShards.size == 1) + assert(splitOpenShards.size == 2) + + eventually(timeout(60 seconds), interval(10 second)) { + localTestUtils.pushData(testData2, aggregateTestData) + assert(collected.synchronized { collected === (testData1 ++ testData2).toSet }, + "\nData received does not match data sent after splitting a shard") + } + + val Seq(shardToMerge, adjShard) = splitOpenShards + localTestUtils.mergeShard(shardToMerge.getShardId, adjShard.getShardId) + val (mergedOpenShards, mergedCloseShards) = localTestUtils.getShards().partition { shard => + shard.getSequenceNumberRange.getEndingSequenceNumber == null + } + + // We should have three closed shards and one open shard + assert(mergedCloseShards.size == 3) + assert(mergedOpenShards.size == 1) + + eventually(timeout(60 seconds), interval(10 second)) { + localTestUtils.pushData(testData3, aggregateTestData) + assert(collected.synchronized { collected === (testData1 ++ testData2 ++ testData3).toSet }, + "\nData received does not match data sent after merging shards") + } + } finally { + ssc.stop(stopSparkContext = false) + localTestUtils.deleteStream() + localTestUtils.deleteDynamoDBTable(localAppName) + } + } + testIfEnabled("failure recovery") { val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) val checkpointDir = Utils.createTempDir().getAbsolutePath diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index c55a5885ba805..b3a3420b8494d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -331,7 +331,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab /** * Merges multiple edges between two vertices into a single edge. For correct results, the graph - * must have been partitioned using [[partitionBy]]. + * must have been partitioned using `partitionBy`. * * @param merge the user-supplied commutative associative function to merge edge attributes * for duplicate edges. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 90907300be975..475bccf9bfc76 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -428,7 +428,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Compute the connected component membership of each vertex and return a graph with the vertex * value containing the lowest vertex id in the connected component containing that vertex. * - * @see [[org.apache.spark.graphx.lib.ConnectedComponents$#run]] + * @see `org.apache.spark.graphx.lib.ConnectedComponents.run` */ def connectedComponents(): Graph[VertexId, ED] = { ConnectedComponents.run(graph) @@ -438,7 +438,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Compute the connected component membership of each vertex and return a graph with the vertex * value containing the lowest vertex id in the connected component containing that vertex. * - * @see [[org.apache.spark.graphx.lib.ConnectedComponents$#run]] + * @see `org.apache.spark.graphx.lib.ConnectedComponents.run` */ def connectedComponents(maxIterations: Int): Graph[VertexId, ED] = { ConnectedComponents.run(graph, maxIterations) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala index f1ecc9e2219d1..7a24e320c3e04 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.graphx import org.apache.spark.SparkFunSuite import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { @@ -33,4 +34,30 @@ class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { } } + test("checkpointing") { + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) + edges.checkpoint() + + // EdgeRDD not yet checkpointed + assert(!edges.isCheckpointed) + assert(!edges.isCheckpointedAndMaterialized) + assert(!edges.partitionsRDD.isCheckpointed) + assert(!edges.partitionsRDD.isCheckpointedAndMaterialized) + + val data = edges.collect().toSeq // force checkpointing + + // EdgeRDD shows up as checkpointed, but internally it is not. + // Only internal partitionsRDD is checkpointed. + assert(edges.isCheckpointed) + assert(!edges.isCheckpointedAndMaterialized) + assert(edges.partitionsRDD.isCheckpointed) + assert(edges.partitionsRDD.isCheckpointedAndMaterialized) + + assert(edges.collect().toSeq === data) // test checkpointed RDD + } + } + } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 0bb9e0a3ea180..8e630435279de 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx import org.apache.spark.{HashPartitioner, SparkContext, SparkFunSuite} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class VertexRDDSuite extends SparkFunSuite with LocalSparkContext { @@ -197,4 +198,29 @@ class VertexRDDSuite extends SparkFunSuite with LocalSparkContext { } } + test("checkpoint") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) + verts.checkpoint() + + // VertexRDD not yet checkpointed + assert(!verts.isCheckpointed) + assert(!verts.isCheckpointedAndMaterialized) + assert(!verts.partitionsRDD.isCheckpointed) + assert(!verts.partitionsRDD.isCheckpointedAndMaterialized) + + val data = verts.collect().toSeq // force checkpointing + + // VertexRDD shows up as checkpointed, but internally it is not. + // Only internal partitionsRDD is checkpointed. + assert(verts.isCheckpointed) + assert(!verts.isCheckpointedAndMaterialized) + assert(verts.partitionsRDD.isCheckpointed) + assert(verts.partitionsRDD.isCheckpointedAndMaterialized) + + assert(verts.collect().toSeq === data) // test checkpointed RDD + } + } } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java index 3ee5b8cf9689d..3bc35da7cc27c 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java @@ -23,11 +23,8 @@ import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; import static org.mockito.Mockito.*; -import static org.apache.spark.launcher.SparkSubmitOptionParser.*; - public class SparkSubmitOptionParserSuite extends BaseSuite { private SparkSubmitOptionParser parser; diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index 08e9cb9ba8668..b76dc5f93193c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -83,11 +83,11 @@ abstract class PipelineStage extends Params with Logging { /** * A simple pipeline, which acts as an estimator. A Pipeline consists of a sequence of stages, each - * of which is either an [[Estimator]] or a [[Transformer]]. When [[Pipeline#fit]] is called, the - * stages are executed in order. If a stage is an [[Estimator]], its [[Estimator#fit]] method will + * of which is either an [[Estimator]] or a [[Transformer]]. When `Pipeline.fit` is called, the + * stages are executed in order. If a stage is an [[Estimator]], its `Estimator.fit` method will * be called on the input dataset to fit a model. Then the model, which is a transformer, will be * used to transform the dataset as the input to the next stage. If a stage is a [[Transformer]], - * its [[Transformer#transform]] method will be called to produce the dataset for the next stage. + * its `Transformer.transform` method will be called to produce the dataset for the next stage. * The fitted model from a [[Pipeline]] is a [[PipelineModel]], which consists of fitted models and * transformers, corresponding to the pipeline stages. If there are no stages, the pipeline acts as * an identity transformer. @@ -121,9 +121,9 @@ class Pipeline @Since("1.4.0") ( /** * Fits the pipeline to the input dataset with additional parameters. If a stage is an - * [[Estimator]], its [[Estimator#fit]] method will be called on the input dataset to fit a model. + * [[Estimator]], its `Estimator.fit` method will be called on the input dataset to fit a model. * Then the model, which is a transformer, will be used to transform the dataset as the input to - * the next stage. If a stage is a [[Transformer]], its [[Transformer#transform]] method will be + * the next stage. If a stage is a [[Transformer]], its `Transformer.transform` method will be * called to produce the dataset for the next stage. The fitted model from a [[Pipeline]] is an * [[PipelineModel]], which consists of fitted models and transformers, corresponding to the * pipeline stages. If there are no stages, the output model acts as an identity transformer. diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index 4b43a3aa5b700..215f9d86f1be1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -40,7 +40,7 @@ private[ml] trait PredictorParams extends Params * @param schema input schema * @param fitting whether this is in fitting * @param featuresDataType SQL DataType for FeaturesType. - * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * E.g., [[VectorUDT]] for vector features. * @return output schema */ protected def validateAndTransformSchema( @@ -51,6 +51,14 @@ private[ml] trait PredictorParams extends Params SchemaUtils.checkColumnType(schema, $(featuresCol), featuresDataType) if (fitting) { SchemaUtils.checkNumericType(schema, $(labelCol)) + + this match { + case p: HasWeightCol => + if (isDefined(p.weightCol) && $(p.weightCol).nonEmpty) { + SchemaUtils.checkNumericType(schema, $(p.weightCol)) + } + case _ => + } } SchemaUtils.appendColumn(schema, $(predictionCol), DoubleType) } @@ -59,10 +67,12 @@ private[ml] trait PredictorParams extends Params /** * :: DeveloperApi :: * Abstraction for prediction problems (regression and classification). It accepts all NumericType - * labels and will automatically cast it to DoubleType in `fit()`. + * labels and will automatically cast it to DoubleType in `fit()`. If this predictor supports + * weights, it accepts all NumericType weights, which will be automatically casted to DoubleType + * in `fit()`. * * @tparam FeaturesType Type of features. - * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * E.g., [[VectorUDT]] for vector features. * @tparam Learner Specialization of this class. If you subclass this type, use this type * parameter to specify the concrete type. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type @@ -91,7 +101,19 @@ abstract class Predictor[ // Cast LabelCol to DoubleType and keep the metadata. val labelMeta = dataset.schema($(labelCol)).metadata - val casted = dataset.withColumn($(labelCol), col($(labelCol)).cast(DoubleType), labelMeta) + val labelCasted = dataset.withColumn($(labelCol), col($(labelCol)).cast(DoubleType), labelMeta) + + // Cast WeightCol to DoubleType and keep the metadata. + val casted = this match { + case p: HasWeightCol => + if (isDefined(p.weightCol) && $(p.weightCol).nonEmpty) { + val weightMeta = dataset.schema($(p.weightCol)).metadata + labelCasted.withColumn($(p.weightCol), col($(p.weightCol)).cast(DoubleType), weightMeta) + } else { + labelCasted + } + case _ => labelCasted + } copyValues(train(casted).setParent(this)) } @@ -138,7 +160,7 @@ abstract class Predictor[ * Abstraction for a model for prediction tasks (regression and classification). * * @tparam FeaturesType Type of features. - * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * E.g., [[VectorUDT]] for vector features. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index a3da3067e1b5f..8fd6b70d4ddb4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -27,7 +27,7 @@ import org.apache.spark.ml.util.{MetadataUtils, SchemaUtils} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DataType, DoubleType, StructType} +import org.apache.spark.sql.types.{DataType, StructType} /** * (private[spark]) Params for classification. diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index c99b63b25d2e7..ade0960f87a0d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -23,9 +23,8 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.feature.LabeledPoint -import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ @@ -33,11 +32,11 @@ import org.apache.spark.ml.tree.impl.GradientBoostedTrees import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.tree.loss.LogLoss import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType /** * Gradient-Boosted Trees (GBTs) (http://en.wikipedia.org/wiki/Gradient_boosting) @@ -59,7 +58,7 @@ import org.apache.spark.sql.types.DoubleType @Since("1.4.0") class GBTClassifier @Since("1.4.0") ( @Since("1.4.0") override val uid: String) - extends Predictor[Vector, GBTClassifier, GBTClassificationModel] + extends ProbabilisticClassifier[Vector, GBTClassifier, GBTClassificationModel] with GBTClassifierParams with DefaultParamsWritable with Logging { @Since("1.4.0") @@ -159,10 +158,19 @@ class GBTClassifier @Since("1.4.0") ( val numFeatures = oldDataset.first().features.size val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) + val numClasses = 2 + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + val instr = Instrumentation.create(this, oldDataset) - instr.logParams(params: _*) + instr.logParams(labelCol, featuresCol, predictionCol, impurity, lossType, + maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, + seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval) instr.logNumFeatures(numFeatures) - instr.logNumClasses(2) + instr.logNumClasses(numClasses) val (baseLearners, learnerWeights) = GradientBoostedTrees.run(oldDataset, boostingStrategy, $(seed)) @@ -201,8 +209,9 @@ class GBTClassificationModel private[ml]( @Since("1.6.0") override val uid: String, private val _trees: Array[DecisionTreeRegressionModel], private val _treeWeights: Array[Double], - @Since("1.6.0") override val numFeatures: Int) - extends PredictionModel[Vector, GBTClassificationModel] + @Since("1.6.0") override val numFeatures: Int, + @Since("2.2.0") override val numClasses: Int) + extends ProbabilisticClassificationModel[Vector, GBTClassificationModel] with GBTClassifierParams with TreeEnsembleModel[DecisionTreeRegressionModel] with MLWritable with Serializable { @@ -210,6 +219,20 @@ class GBTClassificationModel private[ml]( require(_trees.length == _treeWeights.length, "GBTClassificationModel given trees, treeWeights" + s" of non-matching lengths (${_trees.length}, ${_treeWeights.length}, respectively).") + /** + * Construct a GBTClassificationModel + * + * @param _trees Decision trees in the ensemble. + * @param _treeWeights Weights for the decision trees in the ensemble. + * @param numFeatures The number of features. + */ + private[ml] def this( + uid: String, + _trees: Array[DecisionTreeRegressionModel], + _treeWeights: Array[Double], + numFeatures: Int) = + this(uid, _trees, _treeWeights, numFeatures, 2) + /** * Construct a GBTClassificationModel * @@ -218,7 +241,7 @@ class GBTClassificationModel private[ml]( */ @Since("1.6.0") def this(uid: String, _trees: Array[DecisionTreeRegressionModel], _treeWeights: Array[Double]) = - this(uid, _trees, _treeWeights, -1) + this(uid, _trees, _treeWeights, -1, 2) @Since("1.4.0") override def trees: Array[DecisionTreeRegressionModel] = _trees @@ -241,11 +264,29 @@ class GBTClassificationModel private[ml]( } override protected def predict(features: Vector): Double = { - // TODO: When we add a generic Boosting class, handle transform there? SPARK-7129 - // Classifies by thresholding sum of weighted tree predictions - val treePredictions = _trees.map(_.rootNode.predictImpl(features).prediction) - val prediction = blas.ddot(numTrees, treePredictions, 1, _treeWeights, 1) - if (prediction > 0.0) 1.0 else 0.0 + // If thresholds defined, use predictRaw to get probabilities, otherwise use optimization + if (isDefined(thresholds)) { + super.predict(features) + } else { + if (margin(features) > 0.0) 1.0 else 0.0 + } + } + + override protected def predictRaw(features: Vector): Vector = { + val prediction: Double = margin(features) + Vectors.dense(Array(-prediction, prediction)) + } + + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { + rawPrediction match { + case dv: DenseVector => + dv.values(0) = loss.computeProbability(dv.values(0)) + dv.values(1) = 1.0 - dv.values(0) + dv + case sv: SparseVector => + throw new RuntimeException("Unexpected error in GBTClassificationModel:" + + " raw2probabilityInPlace encountered SparseVector") + } } /** Number of trees in ensemble */ @@ -253,7 +294,7 @@ class GBTClassificationModel private[ml]( @Since("1.4.0") override def copy(extra: ParamMap): GBTClassificationModel = { - copyValues(new GBTClassificationModel(uid, _trees, _treeWeights, numFeatures), + copyValues(new GBTClassificationModel(uid, _trees, _treeWeights, numFeatures, numClasses), extra).setParent(parent) } @@ -275,11 +316,20 @@ class GBTClassificationModel private[ml]( @Since("2.0.0") lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(trees, numFeatures) + /** Raw prediction for the positive class. */ + private def margin(features: Vector): Double = { + val treePredictions = _trees.map(_.rootNode.predictImpl(features).prediction) + blas.ddot(numTrees, treePredictions, 1, _treeWeights, 1) + } + /** (private[ml]) Convert to a model in the old API */ private[ml] def toOld: OldGBTModel = { new OldGBTModel(OldAlgo.Classification, _trees.map(_.toOld), _treeWeights) } + // hard coded loss, which is not meant to be changed in the model + private val loss = getOldLossType + @Since("2.0.0") override def write: MLWriter = new GBTClassificationModel.GBTClassificationModelWriter(this) } @@ -287,6 +337,9 @@ class GBTClassificationModel private[ml]( @Since("2.0.0") object GBTClassificationModel extends MLReadable[GBTClassificationModel] { + private val numFeaturesKey: String = "numFeatures" + private val numTreesKey: String = "numTrees" + @Since("2.0.0") override def read: MLReader[GBTClassificationModel] = new GBTClassificationModelReader @@ -299,8 +352,8 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { override protected def saveImpl(path: String): Unit = { val extraMetadata: JObject = Map( - "numFeatures" -> instance.numFeatures, - "numTrees" -> instance.getNumTrees) + numFeaturesKey -> instance.numFeatures, + numTreesKey -> instance.getNumTrees) EnsembleModelReadWrite.saveImpl(instance, path, sparkSession, extraMetadata) } } @@ -315,8 +368,8 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { implicit val format = DefaultFormats val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) - val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] - val numTrees = (metadata.metadata \ "numTrees").extract[Int] + val numFeatures = (metadata.metadata \ numFeaturesKey).extract[Int] + val numTrees = (metadata.metadata \ numTreesKey).extract[Int] val trees: Array[DecisionTreeRegressionModel] = treesData.map { case (treeMetadata, root) => @@ -327,7 +380,8 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { } require(numTrees == trees.length, s"GBTClassificationModel.load expected $numTrees" + s" trees based on metadata but found ${trees.length} trees.") - val model = new GBTClassificationModel(metadata.uid, trees, treeWeights, numFeatures) + val model = new GBTClassificationModel(metadata.uid, + trees, treeWeights, numFeatures) DefaultParamsReader.getAndSetParams(model, metadata) model } @@ -338,7 +392,8 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { oldModel: OldGBTModel, parent: GBTClassifier, categoricalFeatures: Map[Int, Int], - numFeatures: Int = -1): GBTClassificationModel = { + numFeatures: Int = -1, + numClasses: Int = 2): GBTClassificationModel = { require(oldModel.algo == OldAlgo.Classification, "Cannot convert GradientBoostedTreesModel" + s" with algo=${oldModel.algo} (old API) to GBTClassificationModel (new API).") val newTrees = oldModel.trees.map { tree => @@ -346,6 +401,6 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { DecisionTreeRegressionModel.fromOld(tree, null, categoricalFeatures) } val uid = if (parent != null) parent.uid else Identifiable.randomUID("gbtc") - new GBTClassificationModel(uid, newTrees, oldModel.treeWeights, numFeatures) + new GBTClassificationModel(uid, newTrees, oldModel.treeWeights, numFeatures, numClasses) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala new file mode 100644 index 0000000000000..3b14c4b00416a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -0,0 +1,546 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification + +import scala.collection.mutable + +import breeze.linalg.{DenseVector => BDV} +import breeze.optimize.{CachedDiffFunction, DiffFunction, OWLQN => BreezeOWLQN} +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.linalg.BLAS._ +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions.{col, lit} + +/** Params for linear SVM Classifier. */ +private[classification] trait LinearSVCParams extends ClassifierParams with HasRegParam + with HasMaxIter with HasFitIntercept with HasTol with HasStandardization with HasWeightCol + with HasThreshold with HasAggregationDepth + +/** + * :: Experimental :: + * + * Linear SVM Classifier (https://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM) + * + * This binary classifier optimizes the Hinge Loss using the OWLQN optimizer. + * + */ +@Since("2.2.0") +@Experimental +class LinearSVC @Since("2.2.0") ( + @Since("2.2.0") override val uid: String) + extends Classifier[Vector, LinearSVC, LinearSVCModel] + with LinearSVCParams with DefaultParamsWritable { + + @Since("2.2.0") + def this() = this(Identifiable.randomUID("linearsvc")) + + /** + * Set the regularization parameter. + * Default is 0.0. + * + * @group setParam + */ + @Since("2.2.0") + def setRegParam(value: Double): this.type = set(regParam, value) + setDefault(regParam -> 0.0) + + /** + * Set the maximum number of iterations. + * Default is 100. + * + * @group setParam + */ + @Since("2.2.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + setDefault(maxIter -> 100) + + /** + * Whether to fit an intercept term. + * Default is true. + * + * @group setParam + */ + @Since("2.2.0") + def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) + setDefault(fitIntercept -> true) + + /** + * Set the convergence tolerance of iterations. + * Smaller values will lead to higher accuracy at the cost of more iterations. + * Default is 1E-6. + * + * @group setParam + */ + @Since("2.2.0") + def setTol(value: Double): this.type = set(tol, value) + setDefault(tol -> 1E-6) + + /** + * Whether to standardize the training features before fitting the model. + * Default is true. + * + * @group setParam + */ + @Since("2.2.0") + def setStandardization(value: Boolean): this.type = set(standardization, value) + setDefault(standardization -> true) + + /** + * Set the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("2.2.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + + /** + * Set threshold in binary classification, in range [0, 1]. + * + * @group setParam + */ + @Since("2.2.0") + def setThreshold(value: Double): this.type = set(threshold, value) + setDefault(threshold -> 0.0) + + /** + * Suggested depth for treeAggregate (greater than or equal to 2). + * If the dimensions of features or the number of partitions are large, + * this param could be adjusted to a larger size. + * Default is 2. + * + * @group expertSetParam + */ + @Since("2.2.0") + def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) + setDefault(aggregationDepth -> 2) + + @Since("2.2.0") + override def copy(extra: ParamMap): LinearSVC = defaultCopy(extra) + + override protected[classification] def train(dataset: Dataset[_]): LinearSVCModel = { + val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) + val instances: RDD[Instance] = + dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, features: Vector) => + Instance(label, weight, features) + } + + val instr = Instrumentation.create(this, instances) + instr.logParams(regParam, maxIter, fitIntercept, tol, standardization, threshold, + aggregationDepth) + + val (summarizer, labelSummarizer) = { + val seqOp = (c: (MultivariateOnlineSummarizer, MultiClassSummarizer), + instance: Instance) => + (c._1.add(instance.features, instance.weight), c._2.add(instance.label, instance.weight)) + + val combOp = (c1: (MultivariateOnlineSummarizer, MultiClassSummarizer), + c2: (MultivariateOnlineSummarizer, MultiClassSummarizer)) => + (c1._1.merge(c2._1), c1._2.merge(c2._2)) + + instances.treeAggregate( + new MultivariateOnlineSummarizer, new MultiClassSummarizer + )(seqOp, combOp, $(aggregationDepth)) + } + + val histogram = labelSummarizer.histogram + val numInvalid = labelSummarizer.countInvalid + val numFeatures = summarizer.mean.size + val numFeaturesPlusIntercept = if (getFitIntercept) numFeatures + 1 else numFeatures + + val numClasses = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { + case Some(n: Int) => + require(n >= histogram.length, s"Specified number of classes $n was " + + s"less than the number of unique labels ${histogram.length}.") + n + case None => histogram.length + } + require(numClasses == 2, s"LinearSVC only supports binary classification." + + s" $numClasses classes detected in $labelCol") + instr.logNumClasses(numClasses) + instr.logNumFeatures(numFeatures) + + val (coefficientVector, interceptVector, objectiveHistory) = { + if (numInvalid != 0) { + val msg = s"Classification labels should be in [0 to ${numClasses - 1}]. " + + s"Found $numInvalid invalid labels." + logError(msg) + throw new SparkException(msg) + } + + val featuresStd = summarizer.variance.toArray.map(math.sqrt) + val regParamL2 = $(regParam) + val bcFeaturesStd = instances.context.broadcast(featuresStd) + val costFun = new LinearSVCCostFun(instances, $(fitIntercept), + $(standardization), bcFeaturesStd, regParamL2, $(aggregationDepth)) + + def regParamL1Fun = (index: Int) => 0D + val optimizer = new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, regParamL1Fun, $(tol)) + val initialCoefWithIntercept = Vectors.zeros(numFeaturesPlusIntercept) + + val states = optimizer.iterations(new CachedDiffFunction(costFun), + initialCoefWithIntercept.asBreeze.toDenseVector) + + val scaledObjectiveHistory = mutable.ArrayBuilder.make[Double] + var state: optimizer.State = null + while (states.hasNext) { + state = states.next() + scaledObjectiveHistory += state.adjustedValue + } + + bcFeaturesStd.destroy(blocking = false) + if (state == null) { + val msg = s"${optimizer.getClass.getName} failed." + logError(msg) + throw new SparkException(msg) + } + + /* + The coefficients are trained in the scaled space; we're converting them back to + the original space. + Note that the intercept in scaled space and original space is the same; + as a result, no scaling is needed. + */ + val rawCoefficients = state.x.toArray + val coefficientArray = Array.tabulate(numFeatures) { i => + if (featuresStd(i) != 0.0) { + rawCoefficients(i) / featuresStd(i) + } else { + 0.0 + } + } + + val intercept = if ($(fitIntercept)) { + rawCoefficients(numFeaturesPlusIntercept - 1) + } else { + 0.0 + } + (Vectors.dense(coefficientArray), intercept, scaledObjectiveHistory.result()) + } + + val model = copyValues(new LinearSVCModel(uid, coefficientVector, interceptVector)) + instr.logSuccess(model) + model + } +} + +@Since("2.2.0") +object LinearSVC extends DefaultParamsReadable[LinearSVC] { + + @Since("2.2.0") + override def load(path: String): LinearSVC = super.load(path) +} + +/** + * :: Experimental :: + * SVM Model trained by [[LinearSVC]] + */ +@Since("2.2.0") +@Experimental +class LinearSVCModel private[classification] ( + @Since("2.2.0") override val uid: String, + @Since("2.2.0") val coefficients: Vector, + @Since("2.2.0") val intercept: Double) + extends ClassificationModel[Vector, LinearSVCModel] + with LinearSVCParams with MLWritable { + + @Since("2.2.0") + override val numClasses: Int = 2 + + @Since("2.2.0") + override val numFeatures: Int = coefficients.size + + @Since("2.2.0") + def setThreshold(value: Double): this.type = set(threshold, value) + + @Since("2.2.0") + def setWeightCol(value: Double): this.type = set(threshold, value) + + private val margin: Vector => Double = (features) => { + BLAS.dot(features, coefficients) + intercept + } + + override protected def predict(features: Vector): Double = { + if (margin(features) > $(threshold)) 1.0 else 0.0 + } + + override protected def predictRaw(features: Vector): Vector = { + val m = margin(features) + Vectors.dense(-m, m) + } + + @Since("2.2.0") + override def copy(extra: ParamMap): LinearSVCModel = { + copyValues(new LinearSVCModel(uid, coefficients, intercept), extra).setParent(parent) + } + + @Since("2.2.0") + override def write: MLWriter = new LinearSVCModel.LinearSVCWriter(this) + +} + + +@Since("2.2.0") +object LinearSVCModel extends MLReadable[LinearSVCModel] { + + @Since("2.2.0") + override def read: MLReader[LinearSVCModel] = new LinearSVCReader + + @Since("2.2.0") + override def load(path: String): LinearSVCModel = super.load(path) + + /** [[MLWriter]] instance for [[LinearSVCModel]] */ + private[LinearSVCModel] + class LinearSVCWriter(instance: LinearSVCModel) + extends MLWriter with Logging { + + private case class Data(coefficients: Vector, intercept: Double) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.coefficients, instance.intercept) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class LinearSVCReader extends MLReader[LinearSVCModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[LinearSVCModel].getName + + override def load(path: String): LinearSVCModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.format("parquet").load(dataPath) + val Row(coefficients: Vector, intercept: Double) = + data.select("coefficients", "intercept").head() + val model = new LinearSVCModel(metadata.uid, coefficients, intercept) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } +} + +/** + * LinearSVCCostFun implements Breeze's DiffFunction[T] for hinge loss function + */ +private class LinearSVCCostFun( + instances: RDD[Instance], + fitIntercept: Boolean, + standardization: Boolean, + bcFeaturesStd: Broadcast[Array[Double]], + regParamL2: Double, + aggregationDepth: Int) extends DiffFunction[BDV[Double]] { + + override def calculate(coefficients: BDV[Double]): (Double, BDV[Double]) = { + val coeffs = Vectors.fromBreeze(coefficients) + val bcCoeffs = instances.context.broadcast(coeffs) + val featuresStd = bcFeaturesStd.value + val numFeatures = featuresStd.length + + val svmAggregator = { + val seqOp = (c: LinearSVCAggregator, instance: Instance) => c.add(instance) + val combOp = (c1: LinearSVCAggregator, c2: LinearSVCAggregator) => c1.merge(c2) + + instances.treeAggregate( + new LinearSVCAggregator(bcCoeffs, bcFeaturesStd, fitIntercept) + )(seqOp, combOp, aggregationDepth) + } + + val totalGradientArray = svmAggregator.gradient.toArray + // regVal is the sum of coefficients squares excluding intercept for L2 regularization. + val regVal = if (regParamL2 == 0.0) { + 0.0 + } else { + var sum = 0.0 + coeffs.foreachActive { case (index, value) => + // We do not apply regularization to the intercepts + if (index != numFeatures) { + // The following code will compute the loss of the regularization; also + // the gradient of the regularization, and add back to totalGradientArray. + sum += { + if (standardization) { + totalGradientArray(index) += regParamL2 * value + value * value + } else { + if (featuresStd(index) != 0.0) { + // If `standardization` is false, we still standardize the data + // to improve the rate of convergence; as a result, we have to + // perform this reverse standardization by penalizing each component + // differently to get effectively the same objective function when + // the training dataset is not standardized. + val temp = value / (featuresStd(index) * featuresStd(index)) + totalGradientArray(index) += regParamL2 * temp + value * temp + } else { + 0.0 + } + } + } + } + } + 0.5 * regParamL2 * sum + } + bcCoeffs.destroy(blocking = false) + + (svmAggregator.loss + regVal, new BDV(totalGradientArray)) + } +} + +/** + * LinearSVCAggregator computes the gradient and loss for hinge loss function, as used + * in binary classification for instances in sparse or dense vector in an online fashion. + * + * Two LinearSVCAggregator can be merged together to have a summary of loss and gradient of + * the corresponding joint dataset. + * + * This class standardizes feature values during computation using bcFeaturesStd. + * + * @param bcCoefficients The coefficients corresponding to the features. + * @param fitIntercept Whether to fit an intercept term. + * @param bcFeaturesStd The standard deviation values of the features. + */ +private class LinearSVCAggregator( + bcCoefficients: Broadcast[Vector], + bcFeaturesStd: Broadcast[Array[Double]], + fitIntercept: Boolean) extends Serializable { + + private val numFeatures: Int = bcFeaturesStd.value.length + private val numFeaturesPlusIntercept: Int = if (fitIntercept) numFeatures + 1 else numFeatures + private val coefficients: Vector = bcCoefficients.value + private var weightSum: Double = 0.0 + private var lossSum: Double = 0.0 + require(numFeaturesPlusIntercept == coefficients.size, s"Dimension mismatch. Coefficients " + + s"length ${coefficients.size}, FeaturesStd length ${numFeatures}, fitIntercept: $fitIntercept") + + private val coefficientsArray = coefficients match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"coefficients only supports dense vector but got type ${coefficients.getClass}.") + } + private val gradientSumArray = Array.fill[Double](coefficientsArray.length)(0) + + /** + * Add a new training instance to this LinearSVCAggregator, and update the loss and gradient + * of the objective function. + * + * @param instance The instance of data point to be added. + * @return This LinearSVCAggregator object. + */ + def add(instance: Instance): this.type = { + instance match { case Instance(label, weight, features) => + if (weight == 0.0) return this + val localFeaturesStd = bcFeaturesStd.value + val localCoefficients = coefficientsArray + val localGradientSumArray = gradientSumArray + + val dotProduct = { + var sum = 0.0 + features.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + sum += localCoefficients(index) * value / localFeaturesStd(index) + } + } + if (fitIntercept) sum += localCoefficients(numFeaturesPlusIntercept - 1) + sum + } + // Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x))) + // Therefore the gradient is -(2y - 1)*x + val labelScaled = 2 * label - 1.0 + val loss = if (1.0 > labelScaled * dotProduct) { + weight * (1.0 - labelScaled * dotProduct) + } else { + 0.0 + } + + if (1.0 > labelScaled * dotProduct) { + val gradientScale = -labelScaled * weight + features.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + localGradientSumArray(index) += value * gradientScale / localFeaturesStd(index) + } + } + if (fitIntercept) { + localGradientSumArray(localGradientSumArray.length - 1) += gradientScale + } + } + + lossSum += loss + weightSum += weight + this + } + } + + /** + * Merge another LinearSVCAggregator, and update the loss and gradient + * of the objective function. + * (Note that it's in place merging; as a result, `this` object will be modified.) + * + * @param other The other LinearSVCAggregator to be merged. + * @return This LinearSVCAggregator object. + */ + def merge(other: LinearSVCAggregator): this.type = { + if (other.weightSum != 0.0) { + weightSum += other.weightSum + lossSum += other.lossSum + + var i = 0 + val localThisGradientSumArray = this.gradientSumArray + val localOtherGradientSumArray = other.gradientSumArray + val len = localThisGradientSumArray.length + while (i < len) { + localThisGradientSumArray(i) += localOtherGradientSumArray(i) + i += 1 + } + } + this + } + + def loss: Double = { + if (weightSum != 0) { + lossSum / weightSum + } else 0.0 + } + + def gradient: Vector = { + if (weightSum != 0) { + val result = Vectors.dense(gradientSumArray.clone()) + scal(1.0 / weightSum, result) + result + } else Vectors.dense(Array.fill[Double](coefficientsArray.length)(0)) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 5e1d6eec96a3e..d2b0f2a9e15ce 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -233,7 +233,7 @@ class LogisticRegression @Since("1.2.0") ( /** * Set the convergence tolerance of iterations. - * Smaller value will lead to higher accuracy with the cost of more iterations. + * Smaller value will lead to higher accuracy at the cost of more iterations. * Default is 1E-6. * * @group setParam @@ -1431,7 +1431,7 @@ private class LogisticAggregator( private var weightSum = 0.0 private var lossSum = 0.0 - private val gradientSumArray = Array.ofDim[Double](coefficientSize) + private val gradientSumArray = Array.fill[Double](coefficientSize)(0.0D) if (multinomial && numClasses <= 2) { logInfo(s"Multinomial logistic regression for binary classification yields separate " + diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index aaaf7df34576a..93cc1e6f09727 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -232,8 +232,15 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( * @return Fitted model */ override protected def train(dataset: Dataset[_]): MultilayerPerceptronClassificationModel = { + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, predictionCol, layers, maxIter, tol, + blockSize, solver, stepSize, seed) + val myLayers = $(layers) val labels = myLayers.last + instr.logNumClasses(labels) + instr.logNumFeatures(myLayers.head) + val lpData = extractLabeledPoints(dataset) val data = lpData.map(lp => LabelConverter.encodeLabeledPoint(lp, labels)) val topology = FeedForwardTopology.multiLayerPerceptron(myLayers, softmaxOnTop = true) @@ -258,7 +265,10 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( } trainer.setStackSize($(blockSize)) val mlpModel = trainer.train(data) - new MultilayerPerceptronClassificationModel(uid, myLayers, mlpModel.weights) + val model = new MultilayerPerceptronClassificationModel(uid, myLayers, mlpModel.weights) + + instr.logSuccess(model) + model } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index e90040dbf182b..e5713599406e0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -147,7 +147,12 @@ class NaiveBayes @Since("1.5.0") ( } } + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, + probabilityCol, modelType, smoothing, thresholds) + val numFeatures = dataset.select(col($(featuresCol))).head().getAs[Vector](0).size + instr.logNumFeatures(numFeatures) val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) // Aggregates term frequencies per label. @@ -169,6 +174,7 @@ class NaiveBayes @Since("1.5.0") ( }).collect().sortBy(_._1) val numLabels = aggregated.length + instr.logNumClasses(numLabels) val numDocuments = aggregated.map(_._2._1).sum val labelArray = new Array[Double](numLabels) @@ -198,7 +204,9 @@ class NaiveBayes @Since("1.5.0") ( val pi = Vectors.dense(piArray) val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - new NaiveBayesModel(uid, pi, theta).setOldLabels(labelArray) + val model = new NaiveBayesModel(uid, pi, theta).setOldLabels(labelArray) + instr.logSuccess(model) + model } @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 5bbaafeff329f..ce834f1d17e0d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -131,7 +131,9 @@ class RandomForestClassifier @Since("1.4.0") ( super.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, getOldImpurity) val instr = Instrumentation.create(this, oldDataset) - instr.logParams(params: _*) + instr.logParams(labelCol, featuresCol, predictionCol, probabilityCol, rawPredictionCol, + impurity, numTrees, featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, minInfoGain, + minInstancesPerNode, seed, subsamplingRate, thresholds, cacheNodeIds, checkpointInterval) val trees = RandomForest .run(oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index a7bb41379538d..ea2dc6cfd8d31 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -278,7 +278,9 @@ object GaussianMixtureModel extends MLReadable[GaussianMixtureModel] { * While this process is generally guaranteed to converge, it is not guaranteed * to find a global optimum. * - * @note For high-dimensional data (with many features), this algorithm may perform poorly. + * @note This algorithm is limited in its number of features since it requires storing a covariance + * matrix which has size quadratic in the number of features. Even when the number of features does + * not exceed this limit, this algorithm may perform poorly on high-dimensional data. * This is due to high-dimensional data (a) making it difficult to cluster at all (based * on statistical/theoretical arguments) and (b) numerical issues with Gaussian distributions. */ @@ -344,6 +346,9 @@ class GaussianMixture @Since("2.0.0") ( // Extract the number of features. val numFeatures = instances.first().size + require(numFeatures < GaussianMixture.MAX_NUM_FEATURES, s"GaussianMixture cannot handle more " + + s"than ${GaussianMixture.MAX_NUM_FEATURES} features because the size of the covariance" + + s" matrix is quadratic in the number of features.") val instr = Instrumentation.create(this, instances) instr.logParams(featuresCol, predictionCol, probabilityCol, k, maxIter, seed, tol) @@ -391,8 +396,8 @@ class GaussianMixture @Since("2.0.0") ( val (ws, gs) = sc.parallelize(tuples, numPartitions).map { case (mean, cov, weight) => GaussianMixture.updateWeightsAndGaussians(mean, cov, weight, sumWeights) }.collect().unzip - Array.copy(ws.toArray, 0, weights, 0, ws.length) - Array.copy(gs.toArray, 0, gaussians, 0, gs.length) + Array.copy(ws, 0, weights, 0, ws.length) + Array.copy(gs, 0, gaussians, 0, gs.length) } else { var i = 0 while (i < numClusters) { @@ -416,7 +421,7 @@ class GaussianMixture @Since("2.0.0") ( val model = copyValues(new GaussianMixtureModel(uid, weights, gaussianDists)).setParent(this) val summary = new GaussianMixtureSummary(model.transform(dataset), - $(predictionCol), $(probabilityCol), $(featuresCol), $(k)) + $(predictionCol), $(probabilityCol), $(featuresCol), $(k), logLikelihood) model.setSummary(Some(summary)) instr.logSuccess(model) model @@ -486,6 +491,9 @@ class GaussianMixture @Since("2.0.0") ( @Since("2.0.0") object GaussianMixture extends DefaultParamsReadable[GaussianMixture] { + /** Limit number of features such that numFeatures^2^ < Int.MaxValue */ + private[clustering] val MAX_NUM_FEATURES = math.sqrt(Int.MaxValue).toInt + @Since("2.0.0") override def load(path: String): GaussianMixture = super.load(path) @@ -674,6 +682,7 @@ private class ExpectationAggregator( * in `predictions`. * @param featuresCol Name for column of features in `predictions`. * @param k Number of clusters. + * @param logLikelihood Total log-likelihood for this model on the given data. */ @Since("2.0.0") @Experimental @@ -682,7 +691,9 @@ class GaussianMixtureSummary private[clustering] ( predictionCol: String, @Since("2.0.0") val probabilityCol: String, featuresCol: String, - k: Int) extends ClusteringSummary(predictions, predictionCol, featuresCol, k) { + k: Int, + @Since("2.2.0") val logLikelihood: Double) + extends ClusteringSummary(predictions, predictionCol, featuresCol, k) { /** * Probability of each cluster. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 728a883b1ac21..03f4ac5b28e90 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -888,6 +888,12 @@ class LDA @Since("1.6.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): LDAModel = { transformSchema(dataset.schema, logging = true) + + val instr = Instrumentation.create(this, dataset) + instr.logParams(featuresCol, topicDistributionCol, k, maxIter, subsamplingRate, + checkpointInterval, keepLastCheckpoint, optimizeDocConcentration, topicConcentration, + learningDecay, optimizer, learningOffset, seed) + val oldLDA = new OldLDA() .setK($(k)) .setDocConcentration(getOldDocConcentration) @@ -905,7 +911,11 @@ class LDA @Since("1.6.0") ( case m: OldDistributedLDAModel => new DistributedLDAModel(uid, m.vocabSize, m, dataset.sparkSession, None) } - copyValues(newModel).setParent(this) + + instr.logNumFeatures(newModel.vocabSize) + val model = copyValues(newModel).setParent(this) + instr.logSuccess(model) + model } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 9adb0fa618f29..ab0620ca7553a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -728,7 +728,7 @@ trait Params extends Identifiable with Serializable { } /** - * [[extractParamMap]] with no extra values. + * `extractParamMap` with no extra values. */ final def extractParamMap(): ParamMap = { extractParamMap(ParamMap.empty) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala new file mode 100644 index 0000000000000..71712c1c5eec5 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.clustering.{BisectingKMeans, BisectingKMeansModel} +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class BisectingKMeansWrapper private ( + val pipeline: PipelineModel, + val features: Array[String], + val size: Array[Long], + val isLoaded: Boolean = false) extends MLWritable { + private val bisectingKmeansModel: BisectingKMeansModel = + pipeline.stages.last.asInstanceOf[BisectingKMeansModel] + + lazy val coefficients: Array[Double] = bisectingKmeansModel.clusterCenters.flatMap(_.toArray) + + lazy val k: Int = bisectingKmeansModel.getK + + // If the model is loaded from a saved model, cluster is NULL. It is checked on R side + lazy val cluster: DataFrame = bisectingKmeansModel.summary.cluster + + def fitted(method: String): DataFrame = { + if (method == "centers") { + bisectingKmeansModel.summary.predictions.drop(bisectingKmeansModel.getFeaturesCol) + } else if (method == "classes") { + bisectingKmeansModel.summary.cluster + } else { + throw new UnsupportedOperationException( + s"Method (centers or classes) required but $method found.") + } + } + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(bisectingKmeansModel.getFeaturesCol) + } + + override def write: MLWriter = new BisectingKMeansWrapper.BisectingKMeansWrapperWriter(this) +} + +private[r] object BisectingKMeansWrapper extends MLReadable[BisectingKMeansWrapper] { + + def fit( + data: DataFrame, + formula: String, + k: Int, + maxIter: Int, + seed: String, + minDivisibleClusterSize: Double + ): BisectingKMeansWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setFeaturesCol("features") + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + + val bisectingKmeans = new BisectingKMeans() + .setK(k) + .setMaxIter(maxIter) + .setMinDivisibleClusterSize(minDivisibleClusterSize) + .setFeaturesCol(rFormula.getFeaturesCol) + + if (seed != null && seed.length > 0) bisectingKmeans.setSeed(seed.toInt) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, bisectingKmeans)) + .fit(data) + + val bisectingKmeansModel: BisectingKMeansModel = + pipeline.stages.last.asInstanceOf[BisectingKMeansModel] + val size: Array[Long] = bisectingKmeansModel.summary.clusterSizes + + new BisectingKMeansWrapper(pipeline, features, size) + } + + override def read: MLReader[BisectingKMeansWrapper] = new BisectingKMeansWrapperReader + + override def load(path: String): BisectingKMeansWrapper = super.load(path) + + class BisectingKMeansWrapperWriter(instance: BisectingKMeansWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("features" -> instance.features.toSeq) ~ + ("size" -> instance.size.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class BisectingKMeansWrapperReader extends MLReader[BisectingKMeansWrapper] { + + override def load(path: String): BisectingKMeansWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + val size = (rMetadata \ "size").extract[Array[Long]] + new BisectingKMeansWrapper(pipeline, features, size, isLoaded = true) + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala index b708702959829..9a98a8b18b141 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.functions._ private[r] class GaussianMixtureWrapper private ( val pipeline: PipelineModel, val dim: Int, + val logLikelihood: Double, val isLoaded: Boolean = false) extends MLWritable { private val gmm: GaussianMixtureModel = pipeline.stages(1).asInstanceOf[GaussianMixtureModel] @@ -91,7 +92,10 @@ private[r] object GaussianMixtureWrapper extends MLReadable[GaussianMixtureWrapp .setStages(Array(rFormulaModel, gm)) .fit(data) - new GaussianMixtureWrapper(pipeline, dim) + val gmm: GaussianMixtureModel = pipeline.stages(1).asInstanceOf[GaussianMixtureModel] + val logLikelihood: Double = gmm.summary.logLikelihood + + new GaussianMixtureWrapper(pipeline, dim, logLikelihood) } override def read: MLReader[GaussianMixtureWrapper] = new GaussianMixtureWrapperReader @@ -105,7 +109,8 @@ private[r] object GaussianMixtureWrapper extends MLReadable[GaussianMixtureWrapp val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("dim" -> instance.dim) + ("dim" -> instance.dim) ~ + ("logLikelihood" -> instance.logLikelihood) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) @@ -124,7 +129,8 @@ private[r] object GaussianMixtureWrapper extends MLReadable[GaussianMixtureWrapp val rMetadataStr = sc.textFile(rMetadataPath, 1).first() val rMetadata = parse(rMetadataStr) val dim = (rMetadata \ "dim").extract[Int] - new GaussianMixtureWrapper(pipeline, dim, isLoaded = true) + val logLikelihood = (rMetadata \ "logLikelihood").extract[Double] + new GaussianMixtureWrapper(pipeline, dim, logLikelihood, isLoaded = true) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala index b59fe292349bf..c44179281ba1c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala @@ -64,6 +64,8 @@ private[r] object RWrappers extends MLReader[Object] { GBTRegressorWrapper.load(path) case "org.apache.spark.ml.r.GBTClassifierWrapper" => GBTClassifierWrapper.load(path) + case "org.apache.spark.ml.r.BisectingKMeansWrapper" => + BisectingKMeansWrapper.load(path) case _ => throw new SparkException(s"SparkR read.ml does not support load $className") } diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index cdea90ec1a0bc..97c8655298609 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -457,10 +457,12 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] .map { row => Rating(row.getInt(0), row.getInt(1), row.getFloat(2)) } + val instr = Instrumentation.create(this, ratings) - instr.logParams(rank, numUserBlocks, numItemBlocks, implicitPrefs, alpha, - userCol, itemCol, ratingCol, predictionCol, maxIter, - regParam, nonnegative, checkpointInterval, seed) + instr.logParams(rank, numUserBlocks, numItemBlocks, implicitPrefs, alpha, userCol, + itemCol, ratingCol, predictionCol, maxIter, regParam, nonnegative, checkpointInterval, + seed, intermediateStorageLevel, finalStorageLevel) + val (userFactors, itemFactors) = ALS.train(ratings, rank = $(rank), numUserBlocks = $(numUserBlocks), numItemBlocks = $(numItemBlocks), maxIter = $(maxIter), regParam = $(regParam), implicitPrefs = $(implicitPrefs), @@ -1036,14 +1038,12 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { uniqueSrcIdsBuilder += preSrcId var curCount = 1 var i = 1 - var j = 0 while (i < sz) { val srcId = srcIds(i) if (srcId != preSrcId) { uniqueSrcIdsBuilder += srcId dstCountsBuilder += curCount preSrcId = srcId - j += 1 curCount = 0 } curCount += 1 diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index af68e7b9d5809..2f78dd30b3af7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -227,6 +227,12 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) val numFeatures = featuresStd.size + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, censorCol, predictionCol, quantilesCol, + fitIntercept, maxIter, tol, aggregationDepth) + instr.logNamedValue("quantileProbabilities.size", $(quantileProbabilities).length) + instr.logNumFeatures(numFeatures) + if (!$(fitIntercept) && (0 until numFeatures).exists { i => featuresStd(i) == 0.0 && featuresSummarizer.mean(i) != 0.0 }) { logWarning("Fitting AFTSurvivalRegressionModel without intercept on dataset with " + @@ -276,8 +282,10 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S val coefficients = Vectors.dense(rawCoefficients) val intercept = parameters(1) val scale = math.exp(parameters(0)) - val model = new AFTSurvivalRegressionModel(uid, coefficients, intercept, scale) - copyValues(model.setParent(this)) + val model = copyValues(new AFTSurvivalRegressionModel(uid, coefficients, + intercept, scale).setParent(this)) + instr.logSuccess(model) + model } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index f8ab3d3a45a49..08d175cb94442 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -148,7 +148,9 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) val instr = Instrumentation.create(this, oldDataset) - instr.logParams(params: _*) + instr.logParams(labelCol, featuresCol, predictionCol, impurity, lossType, + maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, + seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval) instr.logNumFeatures(numFeatures) val (baseLearners, learnerWeights) = GradientBoostedTrees.run(oldDataset, boostingStrategy, diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 3891ae63a4e1e..c4f41d07800c5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -48,7 +48,7 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam /** * Param for the name of family which is a description of the error distribution * to be used in the model. - * Supported options: "gaussian", "binomial", "poisson" and "gamma". + * Supported options: "gaussian", "binomial", "poisson", "gamma" and "tweedie". * Default is "gaussian". * * @group param @@ -57,16 +57,41 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam final val family: Param[String] = new Param(this, "family", "The name of family which is a description of the error distribution to be used in the " + s"model. Supported options: ${supportedFamilyNames.mkString(", ")}.", - ParamValidators.inArray[String](supportedFamilyNames.toArray)) + (value: String) => supportedFamilyNames.contains(value.toLowerCase)) /** @group getParam */ @Since("2.0.0") def getFamily: String = $(family) + /** + * Param for the power in the variance function of the Tweedie distribution which provides + * the relationship between the variance and mean of the distribution. + * Only applicable for the Tweedie family. + * (see + * Tweedie Distribution (Wikipedia)) + * Supported values: 0 and [1, Inf). + * Note that variance power 0, 1, or 2 corresponds to the Gaussian, Poisson or Gamma + * family, respectively. + * + * @group param + */ + @Since("2.2.0") + final val variancePower: DoubleParam = new DoubleParam(this, "variancePower", + "The power in the variance function of the Tweedie distribution which characterizes " + + "the relationship between the variance and mean of the distribution. " + + "Only applicable for the Tweedie family. Supported values: 0 and [1, Inf).", + (x: Double) => x >= 1.0 || x == 0.0) + + /** @group getParam */ + @Since("2.2.0") + def getVariancePower: Double = $(variancePower) + /** * Param for the name of link function which provides the relationship * between the linear predictor and the mean of the distribution function. * Supported options: "identity", "log", "inverse", "logit", "probit", "cloglog" and "sqrt". + * This is used only when family is not "tweedie". The link function for the "tweedie" family + * must be specified through [[linkPower]]. * * @group param */ @@ -74,12 +99,27 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam final val link: Param[String] = new Param(this, "link", "The name of link function " + "which provides the relationship between the linear predictor and the mean of the " + s"distribution function. Supported options: ${supportedLinkNames.mkString(", ")}", - ParamValidators.inArray[String](supportedLinkNames.toArray)) + (value: String) => supportedLinkNames.contains(value.toLowerCase)) /** @group getParam */ @Since("2.0.0") def getLink: String = $(link) + /** + * Param for the index in the power link function. Only applicable for the Tweedie family. + * Note that link power 0, 1, -1 or 0.5 corresponds to the Log, Identity, Inverse or Sqrt + * link, respectively. + * + * @group param + */ + @Since("2.2.0") + final val linkPower: DoubleParam = new DoubleParam(this, "linkPower", + "The index in the power link function. Only applicable for the Tweedie family.") + + /** @group getParam */ + @Since("2.2.0") + def getLinkPower: Double = $(linkPower) + /** * Param for link prediction (linear predictor) column name. * Default is not set, which means we do not output link prediction. @@ -106,11 +146,27 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam schema: StructType, fitting: Boolean, featuresDataType: DataType): StructType = { - if (isDefined(link)) { - require(supportedFamilyAndLinkPairs.contains( - Family.fromName($(family)) -> Link.fromName($(link))), "Generalized Linear Regression " + - s"with ${$(family)} family does not support ${$(link)} link function.") + if ($(family).toLowerCase == "tweedie") { + if (isSet(link)) { + logWarning("When family is tweedie, use param linkPower to specify link function. " + + "Setting param link will take no effect.") + } + } else { + if (isSet(variancePower)) { + logWarning("When family is not tweedie, setting param variancePower will take no effect.") + } + if (isSet(linkPower)) { + logWarning("When family is not tweedie, use param link to specify link function. " + + "Setting param linkPower will take no effect.") + } + if (isSet(link)) { + require(supportedFamilyAndLinkPairs.contains( + Family.fromParams(this) -> Link.fromParams(this)), + s"Generalized Linear Regression with ${$(family)} family " + + s"does not support ${$(link)} link function.") + } } + val newSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) if (hasLinkPredictionCol) { SchemaUtils.appendColumn(newSchema, $(linkPredictionCol), DoubleType) @@ -128,13 +184,15 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam * Generalized linear model (Wikipedia)) * specified by giving a symbolic description of the linear * predictor (link function) and a description of the error distribution (family). - * It supports "gaussian", "binomial", "poisson" and "gamma" as family. + * It supports "gaussian", "binomial", "poisson", "gamma" and "tweedie" as family. * Valid link functions for each family is listed below. The first link function of each family * is the default one. * - "gaussian" : "identity", "log", "inverse" * - "binomial" : "logit", "probit", "cloglog" * - "poisson" : "log", "identity", "sqrt" * - "gamma" : "inverse", "identity", "log" + * - "tweedie" : power link function specified through "linkPower". The default link power in + * the tweedie family is 1 - variancePower. */ @Experimental @Since("2.0.0") @@ -157,8 +215,29 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val def setFamily(value: String): this.type = set(family, value) setDefault(family -> Gaussian.name) + /** + * Sets the value of param [[variancePower]]. + * Used only when family is "tweedie". + * Default is 0.0, which corresponds to the "gaussian" family. + * + * @group setParam + */ + @Since("2.2.0") + def setVariancePower(value: Double): this.type = set(variancePower, value) + setDefault(variancePower -> 0.0) + + /** + * Sets the value of param [[linkPower]]. + * Used only when family is "tweedie". + * + * @group setParam + */ + @Since("2.2.0") + def setLinkPower(value: Double): this.type = set(linkPower, value) + /** * Sets the value of param [[link]]. + * Used only when family is not "tweedie". * * @group setParam */ @@ -242,15 +321,14 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val def setLinkPredictionCol(value: String): this.type = set(linkPredictionCol, value) override protected def train(dataset: Dataset[_]): GeneralizedLinearRegressionModel = { - val familyObj = Family.fromName($(family)) - val linkObj = if (isDefined(link)) { - Link.fromName($(link)) - } else { - familyObj.defaultLink - } - val familyAndLink = new FamilyAndLink(familyObj, linkObj) + val familyAndLink = FamilyAndLink(this) val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, weightCol, predictionCol, linkPredictionCol, + family, solver, fitIntercept, link, maxIter, regParam, tol) + instr.logNumFeatures(numFeatures) + if (numFeatures > WeightedLeastSquares.MAX_NUM_FEATURES) { val msg = "Currently, GeneralizedLinearRegression only supports number of features" + s" <= ${WeightedLeastSquares.MAX_NUM_FEATURES}. Found $numFeatures in the input dataset." @@ -264,7 +342,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val Instance(label, weight, features) } - if (familyObj == Gaussian && linkObj == Identity) { + val model = if (familyAndLink.family == Gaussian && familyAndLink.link == Identity) { // TODO: Make standardizeFeatures and standardizeLabel configurable. val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), elasticNetParam = 0.0, standardizeFeatures = true, standardizeLabel = true) @@ -274,21 +352,23 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val .setParent(this)) val trainingSummary = new GeneralizedLinearRegressionTrainingSummary(dataset, model, wlsModel.diagInvAtWA.toArray, 1, getSolver) - return model.setSummary(Some(trainingSummary)) + model.setSummary(Some(trainingSummary)) + } else { + // Fit Generalized Linear Model by iteratively reweighted least squares (IRLS). + val initialModel = familyAndLink.initialize(instances, $(fitIntercept), $(regParam)) + val optimizer = new IterativelyReweightedLeastSquares(initialModel, + familyAndLink.reweightFunc, $(fitIntercept), $(regParam), $(maxIter), $(tol)) + val irlsModel = optimizer.fit(instances) + val model = copyValues( + new GeneralizedLinearRegressionModel(uid, irlsModel.coefficients, irlsModel.intercept) + .setParent(this)) + val trainingSummary = new GeneralizedLinearRegressionTrainingSummary(dataset, model, + irlsModel.diagInvAtWA.toArray, irlsModel.numIterations, getSolver) + model.setSummary(Some(trainingSummary)) } - // Fit Generalized Linear Model by iteratively reweighted least squares (IRLS). - val initialModel = familyAndLink.initialize(instances, $(fitIntercept), $(regParam)) - val optimizer = new IterativelyReweightedLeastSquares(initialModel, familyAndLink.reweightFunc, - $(fitIntercept), $(regParam), $(maxIter), $(tol)) - val irlsModel = optimizer.fit(instances) - - val model = copyValues( - new GeneralizedLinearRegressionModel(uid, irlsModel.coefficients, irlsModel.intercept) - .setParent(this)) - val trainingSummary = new GeneralizedLinearRegressionTrainingSummary(dataset, model, - irlsModel.diagInvAtWA.toArray, irlsModel.numIterations, getSolver) - model.setSummary(Some(trainingSummary)) + instr.logSuccess(model) + model } @Since("2.0.0") @@ -301,7 +381,10 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine @Since("2.0.0") override def load(path: String): GeneralizedLinearRegression = super.load(path) - /** Set of family and link pairs that GeneralizedLinearRegression supports. */ + /** + * Set of family (except for tweedie) and link pairs that GeneralizedLinearRegression supports. + * The link function of the Tweedie family is specified through param linkPower. + */ private[regression] lazy val supportedFamilyAndLinkPairs = Set( Gaussian -> Identity, Gaussian -> Log, Gaussian -> Inverse, Binomial -> Logit, Binomial -> Probit, Binomial -> CLogLog, @@ -310,10 +393,12 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine ) /** Set of family names that GeneralizedLinearRegression supports. */ - private[regression] lazy val supportedFamilyNames = supportedFamilyAndLinkPairs.map(_._1.name) + private[regression] lazy val supportedFamilyNames = + supportedFamilyAndLinkPairs.map(_._1.name).toArray :+ "tweedie" /** Set of link names that GeneralizedLinearRegression supports. */ - private[regression] lazy val supportedLinkNames = supportedFamilyAndLinkPairs.map(_._2.name) + private[regression] lazy val supportedLinkNames = + supportedFamilyAndLinkPairs.map(_._2.name).toArray private[regression] val epsilon: Double = 1E-16 @@ -362,6 +447,24 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine } } + private[regression] object FamilyAndLink { + + /** + * Constructs the FamilyAndLink object from a parameter map + */ + def apply(params: GeneralizedLinearRegressionBase): FamilyAndLink = { + val familyObj = Family.fromParams(params) + val linkObj = if ((params.getFamily.toLowerCase != "tweedie" && + params.isSet(params.link)) || (params.getFamily.toLowerCase == "tweedie" && + params.isSet(params.linkPower))) { + Link.fromParams(params) + } else { + familyObj.defaultLink + } + new FamilyAndLink(familyObj, linkObj) + } + } + /** * A description of the error distribution to be used in the model. * @@ -402,27 +505,109 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine private[regression] object Family { /** - * Gets the [[Family]] object from its name. + * Gets the [[Family]] object based on param family and variancePower. + * If param family is set with "gaussian", "binomial", "poisson" or "gamma", + * return the corresponding object directly; otherwise, construct a Tweedie object + * according to variancePower. * - * @param name family name: "gaussian", "binomial", "poisson" or "gamma". + * @param params the parameter map containing family name and variance power */ - def fromName(name: String): Family = { - name match { + def fromParams(params: GeneralizedLinearRegressionBase): Family = { + params.getFamily.toLowerCase match { case Gaussian.name => Gaussian case Binomial.name => Binomial case Poisson.name => Poisson case Gamma.name => Gamma + case "tweedie" => + params.getVariancePower match { + case 0.0 => Gaussian + case 1.0 => Poisson + case 2.0 => Gamma + case others => new Tweedie(others) + } + } + } + } + + /** + * Tweedie exponential family distribution. + * This includes the special cases of Gaussian, Poisson and Gamma. + */ + private[regression] class Tweedie(val variancePower: Double) + extends Family("tweedie") { + + override val defaultLink: Link = new Power(1.0 - variancePower) + + override def initialize(y: Double, weight: Double): Double = { + if (variancePower >= 1.0 && variancePower < 2.0) { + require(y >= 0.0, s"The response variable of $name($variancePower) family " + + s"should be non-negative, but got $y") + } else if (variancePower >= 2.0) { + require(y > 0.0, s"The response variable of $name($variancePower) family " + + s"should be positive, but got $y") + } + if (y == 0) Tweedie.delta else y + } + + override def variance(mu: Double): Double = math.pow(mu, variancePower) + + private def yp(y: Double, mu: Double, p: Double): Double = { + if (p == 0) { + math.log(y / mu) + } else { + (math.pow(y, p) - math.pow(mu, p)) / p + } + } + + override def deviance(y: Double, mu: Double, weight: Double): Double = { + // Force y >= delta for Poisson or compound Poisson + val y1 = if (variancePower >= 1.0 && variancePower < 2.0) { + math.max(y, Tweedie.delta) + } else { + y + } + 2.0 * weight * + (y * yp(y1, mu, 1.0 - variancePower) - yp(y, mu, 2.0 - variancePower)) + } + + override def aic( + predictions: RDD[(Double, Double, Double)], + deviance: Double, + numInstances: Double, + weightSum: Double): Double = { + /* + This depends on the density of the Tweedie distribution. + Only implemented for Gaussian, Poisson and Gamma at this point. + */ + throw new UnsupportedOperationException("No AIC available for the tweedie family") + } + + override def project(mu: Double): Double = { + if (mu < epsilon) { + epsilon + } else if (mu.isInfinity) { + Double.MaxValue + } else { + mu } } } + private[regression] object Tweedie{ + + /** Constant used in initialization and deviance to avoid numerical issues. */ + val delta: Double = 0.1 + } + /** * Gaussian exponential family distribution. * The default link for the Gaussian family is the identity link. */ - private[regression] object Gaussian extends Family("gaussian") { + private[regression] object Gaussian extends Tweedie(0.0) { - val defaultLink: Link = Identity + override val name: String = "gaussian" + + override val defaultLink: Link = Identity override def initialize(y: Double, weight: Double): Double = y @@ -508,18 +693,20 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Poisson exponential family distribution. * The default link for the Poisson family is the log link. */ - private[regression] object Poisson extends Family("poisson") { + private[regression] object Poisson extends Tweedie(1.0) { + + override val name: String = "poisson" - val defaultLink: Link = Log + override val defaultLink: Link = Log override def initialize(y: Double, weight: Double): Double = { require(y >= 0.0, "The response variable of Poisson family " + s"should be non-negative, but got $y") /* Force Poisson mean > 0 to avoid numerical instability in IRLS. - R uses y + 0.1 for initialization. See poisson()$initialize. + R uses y + delta for initialization. See poisson()$initialize. */ - math.max(y, 0.1) + math.max(y, Tweedie.delta) } override def variance(mu: Double): Double = mu @@ -537,25 +724,17 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine weight * dist.Poisson(mu).logProbabilityOf(y.toInt) }.sum() } - - override def project(mu: Double): Double = { - if (mu < epsilon) { - epsilon - } else if (mu.isInfinity) { - Double.MaxValue - } else { - mu - } - } } /** * Gamma exponential family distribution. * The default link for the Gamma family is the inverse link. */ - private[regression] object Gamma extends Family("gamma") { + private[regression] object Gamma extends Tweedie(2.0) { + + override val name: String = "gamma" - val defaultLink: Link = Inverse + override val defaultLink: Link = Inverse override def initialize(y: Double, weight: Double): Double = { require(y > 0.0, "The response variable of Gamma family " + @@ -579,16 +758,6 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine weight * dist.Gamma(1.0 / disp, mu * disp).logPdf(y) }.sum() + 2.0 } - - override def project(mu: Double): Double = { - if (mu < epsilon) { - epsilon - } else if (mu.isInfinity) { - Double.MaxValue - } else { - mu - } - } } /** @@ -613,25 +782,67 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine private[regression] object Link { /** - * Gets the [[Link]] object from its name. + * Gets the [[Link]] object based on param family, link and linkPower. + * If param family is set with "tweedie", return or construct link function object + * according to linkPower; otherwise, return link function object according to link. * - * @param name link name: "identity", "logit", "log", - * "inverse", "probit", "cloglog" or "sqrt". + * @param params the parameter map containing family, link and linkPower */ - def fromName(name: String): Link = { - name match { - case Identity.name => Identity - case Logit.name => Logit - case Log.name => Log - case Inverse.name => Inverse - case Probit.name => Probit - case CLogLog.name => CLogLog - case Sqrt.name => Sqrt + def fromParams(params: GeneralizedLinearRegressionBase): Link = { + if (params.getFamily.toLowerCase == "tweedie") { + params.getLinkPower match { + case 0.0 => Log + case 1.0 => Identity + case -1.0 => Inverse + case 0.5 => Sqrt + case others => new Power(others) + } + } else { + params.getLink.toLowerCase match { + case Identity.name => Identity + case Logit.name => Logit + case Log.name => Log + case Inverse.name => Inverse + case Probit.name => Probit + case CLogLog.name => CLogLog + case Sqrt.name => Sqrt + } + } + } + } + + /** Power link function class */ + private[regression] class Power(val linkPower: Double) + extends Link("power") { + + override def link(mu: Double): Double = { + if (linkPower == 0.0) { + math.log(mu) + } else { + math.pow(mu, linkPower) + } + } + + override def deriv(mu: Double): Double = { + if (linkPower == 0.0) { + 1.0 / mu + } else { + linkPower * math.pow(mu, linkPower - 1.0) + } + } + + override def unlink(eta: Double): Double = { + if (linkPower == 0.0) { + math.exp(eta) + } else { + math.pow(eta, 1.0 / linkPower) } } } - private[regression] object Identity extends Link("identity") { + private[regression] object Identity extends Power(1.0) { + + override val name: String = "identity" override def link(mu: Double): Double = mu @@ -649,7 +860,9 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 / (1.0 + math.exp(-1.0 * eta)) } - private[regression] object Log extends Link("log") { + private[regression] object Log extends Power(0.0) { + + override val name: String = "log" override def link(mu: Double): Double = math.log(mu) @@ -658,7 +871,9 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = math.exp(eta) } - private[regression] object Inverse extends Link("inverse") { + private[regression] object Inverse extends Power(-1.0) { + + override val name: String = "inverse" override def link(mu: Double): Double = 1.0 / mu @@ -687,7 +902,9 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 - math.exp(-1.0 * math.exp(eta)) } - private[regression] object Sqrt extends Link("sqrt") { + private[regression] object Sqrt extends Power(0.5) { + + override val name: String = "sqrt" override def link(mu: Double): Double = math.sqrt(mu) @@ -720,13 +937,7 @@ class GeneralizedLinearRegressionModel private[ml] ( import GeneralizedLinearRegression._ - private lazy val familyObj = Family.fromName($(family)) - private lazy val linkObj = if (isDefined(link)) { - Link.fromName($(link)) - } else { - familyObj.defaultLink - } - private lazy val familyAndLink = new FamilyAndLink(familyObj, linkObj) + private lazy val familyAndLink = FamilyAndLink(this) override protected def predict(features: Vector): Double = { val eta = predictLink(features) @@ -905,12 +1116,11 @@ class GeneralizedLinearRegressionSummary private[regression] ( */ @Since("2.0.0") @transient val predictions: DataFrame = model.transform(dataset) - private[regression] lazy val family: Family = Family.fromName(model.getFamily) - private[regression] lazy val link: Link = if (model.isDefined(model.link)) { - Link.fromName(model.getLink) - } else { - family.defaultLink - } + private[regression] lazy val familyLink: FamilyAndLink = FamilyAndLink(model) + + private[regression] lazy val family: Family = familyLink.family + + private[regression] lazy val link: Link = familyLink.link /** Number of instances in DataFrame predictions. */ private[regression] lazy val numInstances: Long = predictions.count() @@ -1037,7 +1247,8 @@ class GeneralizedLinearRegressionSummary private[regression] ( */ @Since("2.0.0") lazy val dispersion: Double = if ( - model.getFamily == Binomial.name || model.getFamily == Poisson.name) { + model.getFamily.toLowerCase == Binomial.name || + model.getFamily.toLowerCase == Poisson.name) { 1.0 } else { val rss = pearsonResiduals.agg(sum(pow(col("pearsonResiduals"), 2.0))).first().getDouble(0) @@ -1140,7 +1351,8 @@ class GeneralizedLinearRegressionTrainingSummary private[regression] ( @Since("2.0.0") lazy val pValues: Array[Double] = { if (isNormalSolver) { - if (model.getFamily == Binomial.name || model.getFamily == Poisson.name) { + if (model.getFamily.toLowerCase == Binomial.name || + model.getFamily.toLowerCase == Poisson.name) { tValues.map { x => 2.0 * (1.0 - dist.Gaussian(0.0, 1.0).cdf(math.abs(x))) } } else { tValues.map { x => diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index c378a99e3c230..a6c29433d7303 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -86,11 +86,8 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures } else { col($(featuresCol)) } - val w = if (hasWeightCol) { - col($(weightCol)) - } else { - lit(1.0) - } + val w = if (hasWeightCol) col($(weightCol)).cast(DoubleType) else lit(1.0) + dataset.select(col($(labelCol)).cast(DoubleType), f, w).rdd.map { case Row(label: Double, feature: Double, weight: Double) => (label, feature, weight) @@ -109,7 +106,7 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures if (fitting) { SchemaUtils.checkNumericType(schema, $(labelCol)) if (hasWeightCol) { - SchemaUtils.checkColumnType(schema, $(weightCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(weightCol)) } else { logInfo("The weight column is not defined. Treat all instance weights as 1.0.") } @@ -171,10 +168,18 @@ class IsotonicRegression @Since("1.5.0") (@Since("1.5.0") override val uid: Stri val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, weightCol, predictionCol, featureIndex, isotonic) + instr.logNumFeatures(1) + val isotonicRegression = new MLlibIsotonicRegression().setIsotonic($(isotonic)) val oldModel = isotonicRegression.run(instances) - copyValues(new IsotonicRegressionModel(uid, oldModel).setParent(this)) + if (handlePersistence) instances.unpersist() + + val model = copyValues(new IsotonicRegressionModel(uid, oldModel).setParent(this)) + instr.logSuccess(model) + model } @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 534ef87ec64ee..2de7e81d8d41e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -91,7 +91,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String setDefault(regParam -> 0.0) /** - * Set if we should fit the intercept + * Set if we should fit the intercept. * Default is true. * * @group setParam @@ -204,6 +204,11 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String Instance(label, weight, features) } + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, weightCol, predictionCol, solver, tol, + elasticNetParam, fitIntercept, maxIter, regParam, standardization, aggregationDepth) + instr.logNumFeatures(numFeatures) + if (($(solver) == "auto" && numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == "normal") { // For low dimensional data, WeightedLeastSquares is more efficient since the @@ -226,7 +231,9 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String model.diagInvAtWA.toArray, model.objectiveHistory) - return lrModel.setSummary(Some(trainingSummary)) + lrModel.setSummary(Some(trainingSummary)) + instr.logSuccess(lrModel) + return lrModel } val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE @@ -251,10 +258,10 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val rawYStd = math.sqrt(ySummarizer.variance(0)) if (rawYStd == 0.0) { if ($(fitIntercept) || yMean == 0.0) { - // If the rawYStd is zero and fitIntercept=true, then the intercept is yMean with + // If the rawYStd==0 and fitIntercept==true, then the intercept is yMean with // zero coefficient; as a result, training is not needed. // Also, if yMean==0 and rawYStd==0, all the coefficients are zero regardless of - // the fitIntercept + // the fitIntercept. if (yMean == 0.0) { logWarning(s"Mean and standard deviation of the label are zero, so the coefficients " + s"and the intercept will all be zero; as a result, training is not needed.") @@ -279,7 +286,10 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String model, Array(0D), Array(0D)) - return model.setSummary(Some(trainingSummary)) + + model.setSummary(Some(trainingSummary)) + instr.logSuccess(model) + return model } else { require($(regParam) == 0.0, "The standard deviation of the label is zero. " + "Model cannot be regularized.") @@ -401,7 +411,10 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String model, Array(0D), objectiveHistory) + model.setSummary(Some(trainingSummary)) + instr.logSuccess(model) + model } @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index ca4a50b825dde..2f524a8c5784d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -122,7 +122,9 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S super.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, getOldImpurity) val instr = Instrumentation.create(this, oldDataset) - instr.logParams(params: _*) + instr.logParams(labelCol, featuresCol, predictionCol, impurity, numTrees, + featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, minInfoGain, + minInstancesPerNode, seed, subsamplingRate, cacheNodeIds, checkpointInterval) val trees = RandomForest .run(oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 89bbc1556c2d8..100b4bbe221c0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.TaskContext import org.apache.spark.ml.feature.LabeledPoint -import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.ml.linalg.{Vectors, VectorUDT} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala index bc3c86a57c85b..8a9dcb486b7bf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala @@ -113,6 +113,8 @@ private[spark] object DecisionTreeMetadata extends Logging { throw new IllegalArgumentException(s"DecisionTree requires size of input RDD > 0, " + s"but was given by empty one.") } + require(numFeatures > 0, s"DecisionTree requires number of features > 0, " + + s"but was given an empty features vector") val numExamples = input.count() val numClasses = strategy.algo match { case Classification => strategy.numClasses diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index a61ea374cbd46..008dd19c2498d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -714,7 +714,7 @@ private[spark] object RandomForest extends Logging { } // For each (feature, split), calculate the gain, and select the best (feature, split). - val (bestSplit, bestSplitStats) = + val splitsAndImpurityInfo = validFeatureSplits.map { case (featureIndexIdx, featureIndex) => val numSplits = binAggregates.metadata.numSplits(featureIndex) if (binAggregates.metadata.isContinuous(featureIndex)) { @@ -828,8 +828,26 @@ private[spark] object RandomForest extends Logging { new CategoricalSplit(featureIndex, categoriesForSplit.toArray, numCategories) (bestFeatureSplit, bestFeatureGainStats) } - }.maxBy(_._2.gain) + } + val (bestSplit, bestSplitStats) = + if (splitsAndImpurityInfo.isEmpty) { + // If no valid splits for features, then this split is invalid, + // return invalid information gain stats. Take any split and continue. + // Splits is empty, so arbitrarily choose to split on any threshold + val dummyFeatureIndex = featuresForNode.map(_.head).getOrElse(0) + val parentImpurityCalculator = binAggregates.getParentImpurityCalculator() + if (binAggregates.metadata.isContinuous(dummyFeatureIndex)) { + (new ContinuousSplit(dummyFeatureIndex, 0), + ImpurityStats.getInvalidImpurityStats(parentImpurityCalculator)) + } else { + val numCategories = binAggregates.metadata.featureArity(dummyFeatureIndex) + (new CategoricalSplit(dummyFeatureIndex, Array(), numCategories), + ImpurityStats.getInvalidImpurityStats(parentImpurityCalculator)) + } + } else { + splitsAndImpurityInfo.maxBy(_._2.gain) + } (bestSplit, bestSplitStats) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index c7a8f76eca84f..5eb707dfe7bc3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -25,7 +25,7 @@ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, BoostingStrategy => OldBoostingStrategy, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.{Entropy => OldEntropy, Gini => OldGini, Impurity => OldImpurity, Variance => OldVariance} -import org.apache.spark.mllib.tree.loss.{AbsoluteError => OldAbsoluteError, LogLoss => OldLogLoss, Loss => OldLoss, SquaredError => OldSquaredError} +import org.apache.spark.mllib.tree.loss.{AbsoluteError => OldAbsoluteError, ClassificationLoss => OldClassificationLoss, LogLoss => OldLogLoss, Loss => OldLoss, SquaredError => OldSquaredError} import org.apache.spark.sql.types.{DataType, DoubleType, StructType} /** @@ -531,7 +531,7 @@ private[ml] trait GBTClassifierParams extends GBTParams with TreeClassifierParam def getLossType: String = $(lossType).toLowerCase /** (private[ml]) Convert new loss to old loss. */ - override private[ml] def getOldLossType: OldLoss = { + override private[ml] def getOldLossType: OldClassificationLoss = { getLossType match { case "logistic" => OldLogLoss case _ => diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala index a2794368db223..7c46f45c59717 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala @@ -96,7 +96,7 @@ private[spark] class Instrumentation[E <: Estimator[_]] private ( } /** - * Logs the successful completion of the training session and the value of the learned model. + * Logs the successful completion of the training session. */ def logSuccess(model: Model[_]): Unit = { log(s"training finished") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala index 336f2fc114309..ae98e24a75681 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -339,10 +339,15 @@ private object BisectingKMeans extends Serializable { assignments.map { case (index, v) => if (divisibleIndices.contains(index)) { val children = Seq(leftChildIndex(index), rightChildIndex(index)) - val selected = children.minBy { child => - KMeans.fastSquaredDistance(newClusterCenters(child), v) + val newClusterChildren = children.filter(newClusterCenters.contains(_)) + if (newClusterChildren.nonEmpty) { + val selected = newClusterChildren.minBy { child => + KMeans.fastSquaredDistance(newClusterCenters(child), v) + } + (selected, v) + } else { + (index, v) } - (selected, v) } else { (index, v) } @@ -372,12 +377,12 @@ private object BisectingKMeans extends Serializable { internalIndex -= 1 val leftIndex = leftChildIndex(rawIndex) val rightIndex = rightChildIndex(rawIndex) - val height = math.sqrt(Seq(leftIndex, rightIndex).map { childIndex => + val indexes = Seq(leftIndex, rightIndex).filter(clusters.contains(_)) + val height = math.sqrt(indexes.map { childIndex => KMeans.fastSquaredDistance(center, clusters(childIndex).center) }.max) - val left = buildSubTree(leftIndex) - val right = buildSubTree(rightIndex) - new ClusteringTreeNode(index, size, center, cost, height, Array(left, right)) + val children = indexes.map(buildSubTree(_)).toArray + new ClusteringTreeNode(index, size, center, cost, height, children) } else { val index = leafIndex leafIndex += 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index 10bd8468b35cf..051ec2404fb6e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -46,7 +46,9 @@ import org.apache.spark.util.Utils * is considered to have occurred. * @param maxIterations Maximum number of iterations allowed. * - * @note For high-dimensional data (with many features), this algorithm may perform poorly. + * @note This algorithm is limited in its number of features since it requires storing a covariance + * matrix which has size quadratic in the number of features. Even when the number of features does + * not exceed this limit, this algorithm may perform poorly on high-dimensional data. * This is due to high-dimensional data (a) making it difficult to cluster at all (based * on statistical/theoretical arguments) and (b) numerical issues with Gaussian distributions. */ @@ -170,6 +172,9 @@ class GaussianMixture private ( // Get length of the input vectors val d = breezeData.first().length + require(d < GaussianMixture.MAX_NUM_FEATURES, s"GaussianMixture cannot handle more " + + s"than ${GaussianMixture.MAX_NUM_FEATURES} features because the size of the covariance" + + s" matrix is quadratic in the number of features.") val shouldDistributeGaussians = GaussianMixture.shouldDistributeGaussians(k, d) @@ -211,8 +216,8 @@ class GaussianMixture private ( val (ws, gs) = sc.parallelize(tuples, numPartitions).map { case (mean, sigma, weight) => updateWeightsAndGaussians(mean, sigma, weight, sumWeights) }.collect().unzip - Array.copy(ws.toArray, 0, weights, 0, ws.length) - Array.copy(gs.toArray, 0, gaussians, 0, gs.length) + Array.copy(ws, 0, weights, 0, ws.length) + Array.copy(gs, 0, gaussians, 0, gs.length) } else { var i = 0 while (i < k) { @@ -272,6 +277,10 @@ class GaussianMixture private ( } private[clustering] object GaussianMixture { + + /** Limit number of features such that numFeatures^2^ < Int.MaxValue */ + private[clustering] val MAX_NUM_FEATURES = math.sqrt(Int.MaxValue).toInt + /** * Heuristic to distribute the computation of the `MultivariateGaussian`s, approximately when * d is greater than 25 except for when k is very small. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index ae336982092d6..7fd722a332923 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -237,7 +237,7 @@ class LocalLDAModel private[spark] ( vocabSize) /** - * Java-friendly version of [[logLikelihood]] + * Java-friendly version of `logLikelihood` */ @Since("1.5.0") def logLikelihood(documents: JavaPairRDD[java.lang.Long, Vector]): Double = { @@ -259,7 +259,9 @@ class LocalLDAModel private[spark] ( -logLikelihood(documents) / corpusTokenCount } - /** Java-friendly version of [[logPerplexity]] */ + /** + * Java-friendly version of `logPerplexity` + */ @Since("1.5.0") def logPerplexity(documents: JavaPairRDD[java.lang.Long, Vector]): Double = { logPerplexity(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) @@ -365,7 +367,9 @@ class LocalLDAModel private[spark] ( } } - /** Get a method usable as a UDF for [[topicDistributions()]] */ + /** + * Get a method usable as a UDF for `topicDistributions()` + */ private[spark] def getTopicDistributionMethod(sc: SparkContext): Vector => Vector = { val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t) val expElogbetaBc = sc.broadcast(expElogbeta) @@ -414,7 +418,7 @@ class LocalLDAModel private[spark] ( } /** - * Java-friendly version of [[topicDistributions]] + * Java-friendly version of `topicDistributions` */ @Since("1.4.1") def topicDistributions( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index dca031477d3b7..85a90fa959f93 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -80,7 +80,9 @@ class AssociationRules private[fpm] ( }.filter(_.confidence >= minConfidence) } - /** Java-friendly version of [[run]]. */ + /** + * Java-friendly version of `run`. + */ @Since("1.5.0") def run[Item](freqItemsets: JavaRDD[FreqItemset[Item]]): JavaRDD[Rule[Item]] = { val tag = fakeClassTag[Item] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index e3cf0d4979ed4..635da00b6990e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -218,7 +218,9 @@ class FPGrowth private ( new FPGrowthModel(freqItemsets) } - /** Java-friendly version of [[run]]. */ + /** + * Java-friendly version of `run`. + */ @Since("1.3.0") def run[Item, Basket <: JavaIterable[Item]](data: JavaRDD[Basket]): FPGrowthModel[Item] = { implicit val tag = fakeClassTag[Item] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index ff81a2f03e2a8..20d68a34bf3ea 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -425,22 +425,27 @@ class BlockMatrix @Since("1.3.0") ( */ private[distributed] def simulateMultiply( other: BlockMatrix, - partitioner: GridPartitioner): (BlockDestinations, BlockDestinations) = { - val leftMatrix = blockInfo.keys.collect() // blockInfo should already be cached - val rightMatrix = other.blocks.keys.collect() + partitioner: GridPartitioner, + midDimSplitNum: Int): (BlockDestinations, BlockDestinations) = { + val leftMatrix = blockInfo.keys.collect() + val rightMatrix = other.blockInfo.keys.collect() val rightCounterpartsHelper = rightMatrix.groupBy(_._1).mapValues(_.map(_._2)) val leftDestinations = leftMatrix.map { case (rowIndex, colIndex) => val rightCounterparts = rightCounterpartsHelper.getOrElse(colIndex, Array.empty[Int]) val partitions = rightCounterparts.map(b => partitioner.getPartition((rowIndex, b))) - ((rowIndex, colIndex), partitions.toSet) + val midDimSplitIndex = colIndex % midDimSplitNum + ((rowIndex, colIndex), + partitions.toSet.map((pid: Int) => pid * midDimSplitNum + midDimSplitIndex)) }.toMap val leftCounterpartsHelper = leftMatrix.groupBy(_._2).mapValues(_.map(_._1)) val rightDestinations = rightMatrix.map { case (rowIndex, colIndex) => val leftCounterparts = leftCounterpartsHelper.getOrElse(rowIndex, Array.empty[Int]) val partitions = leftCounterparts.map(b => partitioner.getPartition((b, colIndex))) - ((rowIndex, colIndex), partitions.toSet) + val midDimSplitIndex = rowIndex % midDimSplitNum + ((rowIndex, colIndex), + partitions.toSet.map((pid: Int) => pid * midDimSplitNum + midDimSplitIndex)) }.toMap (leftDestinations, rightDestinations) @@ -459,14 +464,39 @@ class BlockMatrix @Since("1.3.0") ( */ @Since("1.3.0") def multiply(other: BlockMatrix): BlockMatrix = { + multiply(other, 1) + } + + /** + * Left multiplies this [[BlockMatrix]] to `other`, another [[BlockMatrix]]. The `colsPerBlock` + * of this matrix must equal the `rowsPerBlock` of `other`. If `other` contains + * `SparseMatrix`, they will have to be converted to a `DenseMatrix`. The output + * [[BlockMatrix]] will only consist of blocks of `DenseMatrix`. This may cause + * some performance issues until support for multiplying two sparse matrices is added. + * Blocks with duplicate indices will be added with each other. + * + * @param other Matrix `B` in `A * B = C` + * @param numMidDimSplits Number of splits to cut on the middle dimension when doing + * multiplication. For example, when multiplying a Matrix `A` of + * size `m x n` with Matrix `B` of size `n x k`, this parameter + * configures the parallelism to use when grouping the matrices. The + * parallelism will increase from `m x k` to `m x k x numMidDimSplits`, + * which in some cases also reduces total shuffled data. + */ + @Since("2.2.0") + def multiply( + other: BlockMatrix, + numMidDimSplits: Int): BlockMatrix = { require(numCols() == other.numRows(), "The number of columns of A and the number of rows " + s"of B must be equal. A.numCols: ${numCols()}, B.numRows: ${other.numRows()}. If you " + "think they should be equal, try setting the dimensions of A and B explicitly while " + "initializing them.") + require(numMidDimSplits > 0, "numMidDimSplits should be a positive integer.") if (colsPerBlock == other.rowsPerBlock) { val resultPartitioner = GridPartitioner(numRowBlocks, other.numColBlocks, math.max(blocks.partitions.length, other.blocks.partitions.length)) - val (leftDestinations, rightDestinations) = simulateMultiply(other, resultPartitioner) + val (leftDestinations, rightDestinations) + = simulateMultiply(other, resultPartitioner, numMidDimSplits) // Each block of A must be multiplied with the corresponding blocks in the columns of B. val flatA = blocks.flatMap { case ((blockRowIndex, blockColIndex), block) => val destinations = leftDestinations.getOrElse((blockRowIndex, blockColIndex), Set.empty) @@ -477,7 +507,11 @@ class BlockMatrix @Since("1.3.0") ( val destinations = rightDestinations.getOrElse((blockRowIndex, blockColIndex), Set.empty) destinations.map(j => (j, (blockRowIndex, blockColIndex, block))) } - val newBlocks = flatA.cogroup(flatB, resultPartitioner).flatMap { case (pId, (a, b)) => + val intermediatePartitioner = new Partitioner { + override def numPartitions: Int = resultPartitioner.numPartitions * numMidDimSplits + override def getPartition(key: Any): Int = key.asInstanceOf[Int] + } + val newBlocks = flatA.cogroup(flatB, intermediatePartitioner).flatMap { case (pId, (a, b)) => a.flatMap { case (leftRowIndex, leftColIndex, leftBlock) => b.filter(_._1 == leftColIndex).map { case (rightRowIndex, rightColIndex, rightBlock) => val C = rightBlock match { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 8979707666a2f..07a67a9e719db 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -279,7 +279,7 @@ object GradientDescent extends Logging { } /** - * Alias of [[runMiniBatchSGD]] with convergenceTol set to default value of 0.001. + * Alias of `runMiniBatchSGD` with convergenceTol set to default value of 0.001. */ def runMiniBatchSGD( data: RDD[(Double, Vector)], diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index 85d4d7f37f2c0..258b1763bba86 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -57,7 +57,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#uniformRDD]]. + * Java-friendly version of `RandomRDDs.uniformRDD`. */ @Since("1.1.0") def uniformJavaRDD( @@ -69,7 +69,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#uniformJavaRDD]] with the default seed. + * `RandomRDDs.uniformJavaRDD` with the default seed. */ @Since("1.1.0") def uniformJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { @@ -77,7 +77,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#uniformJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.uniformJavaRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def uniformJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { @@ -107,7 +107,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#normalRDD]]. + * Java-friendly version of `RandomRDDs.normalRDD`. */ @Since("1.1.0") def normalJavaRDD( @@ -119,7 +119,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#normalJavaRDD]] with the default seed. + * `RandomRDDs.normalJavaRDD` with the default seed. */ @Since("1.1.0") def normalJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { @@ -127,7 +127,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#normalJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.normalJavaRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def normalJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { @@ -157,7 +157,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#poissonRDD]]. + * Java-friendly version of `RandomRDDs.poissonRDD`. */ @Since("1.1.0") def poissonJavaRDD( @@ -170,7 +170,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#poissonJavaRDD]] with the default seed. + * `RandomRDDs.poissonJavaRDD` with the default seed. */ @Since("1.1.0") def poissonJavaRDD( @@ -182,7 +182,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#poissonJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.poissonJavaRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def poissonJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = { @@ -212,7 +212,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#exponentialRDD]]. + * Java-friendly version of `RandomRDDs.exponentialRDD`. */ @Since("1.3.0") def exponentialJavaRDD( @@ -225,7 +225,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#exponentialJavaRDD]] with the default seed. + * `RandomRDDs.exponentialJavaRDD` with the default seed. */ @Since("1.3.0") def exponentialJavaRDD( @@ -237,7 +237,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#exponentialJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.exponentialJavaRDD` with the default number of partitions and the default seed. */ @Since("1.3.0") def exponentialJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = { @@ -269,7 +269,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#gammaRDD]]. + * Java-friendly version of `RandomRDDs.gammaRDD`. */ @Since("1.3.0") def gammaJavaRDD( @@ -283,7 +283,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#gammaJavaRDD]] with the default seed. + * `RandomRDDs.gammaJavaRDD` with the default seed. */ @Since("1.3.0") def gammaJavaRDD( @@ -296,7 +296,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#gammaJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.gammaJavaRDD` with the default number of partitions and the default seed. */ @Since("1.3.0") def gammaJavaRDD( @@ -332,7 +332,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#logNormalRDD]]. + * Java-friendly version of `RandomRDDs.logNormalRDD`. */ @Since("1.3.0") def logNormalJavaRDD( @@ -346,7 +346,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#logNormalJavaRDD]] with the default seed. + * `RandomRDDs.logNormalJavaRDD` with the default seed. */ @Since("1.3.0") def logNormalJavaRDD( @@ -359,7 +359,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#logNormalJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.logNormalJavaRDD` with the default number of partitions and the default seed. */ @Since("1.3.0") def logNormalJavaRDD( @@ -419,7 +419,7 @@ object RandomRDDs { /** * :: DeveloperApi :: - * [[RandomRDDs#randomJavaRDD]] with the default seed. + * `RandomRDDs.randomJavaRDD` with the default seed. */ @DeveloperApi @Since("1.6.0") @@ -433,7 +433,7 @@ object RandomRDDs { /** * :: DeveloperApi :: - * [[RandomRDDs#randomJavaRDD]] with the default seed & numPartitions + * `RandomRDDs.randomJavaRDD` with the default seed & numPartitions */ @DeveloperApi @Since("1.6.0") @@ -469,7 +469,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#uniformVectorRDD]]. + * Java-friendly version of `RandomRDDs.uniformVectorRDD`. */ @Since("1.1.0") def uniformJavaVectorRDD( @@ -482,7 +482,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#uniformJavaVectorRDD]] with the default seed. + * `RandomRDDs.uniformJavaVectorRDD` with the default seed. */ @Since("1.1.0") def uniformJavaVectorRDD( @@ -494,7 +494,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#uniformJavaVectorRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.uniformJavaVectorRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def uniformJavaVectorRDD( @@ -527,7 +527,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#normalVectorRDD]]. + * Java-friendly version of `RandomRDDs.normalVectorRDD`. */ @Since("1.1.0") def normalJavaVectorRDD( @@ -540,7 +540,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#normalJavaVectorRDD]] with the default seed. + * `RandomRDDs.normalJavaVectorRDD` with the default seed. */ @Since("1.1.0") def normalJavaVectorRDD( @@ -552,7 +552,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#normalJavaVectorRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.normalJavaVectorRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def normalJavaVectorRDD( @@ -590,7 +590,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#logNormalVectorRDD]]. + * Java-friendly version of `RandomRDDs.logNormalVectorRDD`. */ @Since("1.3.0") def logNormalJavaVectorRDD( @@ -605,7 +605,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#logNormalJavaVectorRDD]] with the default seed. + * `RandomRDDs.logNormalJavaVectorRDD` with the default seed. */ @Since("1.3.0") def logNormalJavaVectorRDD( @@ -619,7 +619,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#logNormalJavaVectorRDD]] with the default number of partitions and + * `RandomRDDs.logNormalJavaVectorRDD` with the default number of partitions and * the default seed. */ @Since("1.3.0") @@ -657,7 +657,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#poissonVectorRDD]]. + * Java-friendly version of `RandomRDDs.poissonVectorRDD`. */ @Since("1.1.0") def poissonJavaVectorRDD( @@ -671,7 +671,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#poissonJavaVectorRDD]] with the default seed. + * `RandomRDDs.poissonJavaVectorRDD` with the default seed. */ @Since("1.1.0") def poissonJavaVectorRDD( @@ -684,7 +684,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#poissonJavaVectorRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.poissonJavaVectorRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def poissonJavaVectorRDD( @@ -721,7 +721,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#exponentialVectorRDD]]. + * Java-friendly version of `RandomRDDs.exponentialVectorRDD`. */ @Since("1.3.0") def exponentialJavaVectorRDD( @@ -735,7 +735,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#exponentialJavaVectorRDD]] with the default seed. + * `RandomRDDs.exponentialJavaVectorRDD` with the default seed. */ @Since("1.3.0") def exponentialJavaVectorRDD( @@ -748,7 +748,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#exponentialJavaVectorRDD]] with the default number of partitions + * `RandomRDDs.exponentialJavaVectorRDD` with the default number of partitions * and the default seed. */ @Since("1.3.0") @@ -788,7 +788,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#gammaVectorRDD]]. + * Java-friendly version of `RandomRDDs.gammaVectorRDD`. */ @Since("1.3.0") def gammaJavaVectorRDD( @@ -803,7 +803,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#gammaJavaVectorRDD]] with the default seed. + * `RandomRDDs.gammaJavaVectorRDD` with the default seed. */ @Since("1.3.0") def gammaJavaVectorRDD( @@ -817,7 +817,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#gammaJavaVectorRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.gammaJavaVectorRDD` with the default number of partitions and the default seed. */ @Since("1.3.0") def gammaJavaVectorRDD( @@ -857,7 +857,7 @@ object RandomRDDs { /** * :: DeveloperApi :: - * Java-friendly version of [[RandomRDDs#randomVectorRDD]]. + * Java-friendly version of `RandomRDDs.randomVectorRDD`. */ @DeveloperApi @Since("1.6.0") @@ -873,7 +873,7 @@ object RandomRDDs { /** * :: DeveloperApi :: - * [[RandomRDDs#randomJavaVectorRDD]] with the default seed. + * `RandomRDDs.randomJavaVectorRDD` with the default seed. */ @DeveloperApi @Since("1.6.0") @@ -888,7 +888,7 @@ object RandomRDDs { /** * :: DeveloperApi :: - * [[RandomRDDs#randomJavaVectorRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.randomJavaVectorRDD` with the default number of partitions and the default seed. */ @DeveloperApi @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index 36894d52346af..2d236509d5715 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -236,9 +236,8 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { * Only univariate (single feature) algorithm supported. * * Sequential PAV implementation based on: - * Tibshirani, Ryan J., Holger Hoefling, and Robert Tibshirani. - * "Nearly-isotonic regression." Technometrics 53.1 (2011): 54-61. - * Available from here + * Grotzinger, S. J., and C. Witzgall. + * "Projections onto order simplexes." Applied mathematics and Optimization 12.1 (1984): 247-270. * * Sequential PAV parallelization based on: * Kearsley, Anthony J., Richard A. Tapia, and Michael W. Trosset. @@ -312,90 +311,118 @@ class IsotonicRegression private (private var isotonic: Boolean) extends Seriali } /** - * Performs a pool adjacent violators algorithm (PAV). - * Uses approach with single processing of data where violators - * in previously processed data created by pooling are fixed immediately. - * Uses optimization of discovering monotonicity violating sequences (blocks). + * Performs a pool adjacent violators algorithm (PAV). Implements the algorithm originally + * described in [1], using the formulation from [2, 3]. Uses an array to keep track of start + * and end indices of blocks. * - * @param input Input data of tuples (label, feature, weight). + * [1] Grotzinger, S. J., and C. Witzgall. "Projections onto order simplexes." Applied + * mathematics and Optimization 12.1 (1984): 247-270. + * + * [2] Best, Michael J., and Nilotpal Chakravarti. "Active set algorithms for isotonic + * regression; a unifying framework." Mathematical Programming 47.1-3 (1990): 425-439. + * + * [3] Best, Michael J., Nilotpal Chakravarti, and Vasant A. Ubhaya. "Minimizing separable convex + * functions subject to simple chain constraints." SIAM Journal on Optimization 10.3 (2000): + * 658-672. + * + * @param input Input data of tuples (label, feature, weight). Weights must + be non-negative. * @return Result tuples (label, feature, weight) where labels were updated * to form a monotone sequence as per isotonic regression definition. */ private def poolAdjacentViolators( input: Array[(Double, Double, Double)]): Array[(Double, Double, Double)] = { - if (input.isEmpty) { - return Array.empty + val cleanInput = input.filter{ case (y, x, weight) => + require( + weight >= 0.0, + s"Negative weight at point ($y, $x, $weight). Weights must be non-negative" + ) + weight > 0 } - // Pools sub array within given bounds assigning weighted average value to all elements. - def pool(input: Array[(Double, Double, Double)], start: Int, end: Int): Unit = { - val poolSubArray = input.slice(start, end + 1) + if (cleanInput.isEmpty) { + return Array.empty + } - val weightedSum = poolSubArray.map(lp => lp._1 * lp._3).sum - val weight = poolSubArray.map(_._3).sum + // Keeps track of the start and end indices of the blocks. if [i, j] is a valid block from + // cleanInput(i) to cleanInput(j) (inclusive), then blockBounds(i) = j and blockBounds(j) = i + // Initially, each data point is its own block. + val blockBounds = Array.range(0, cleanInput.length) - var i = start - while (i <= end) { - input(i) = (weightedSum / weight, input(i)._2, input(i)._3) - i = i + 1 - } + // Keep track of the sum of weights and sum of weight * y for each block. weights(start) + // gives the values for the block. Entries that are not at the start of a block + // are meaningless. + val weights: Array[(Double, Double)] = cleanInput.map { case (y, _, weight) => + (weight, weight * y) } - var i = 0 - val len = input.length - while (i < len) { - var j = i + // a few convenience functions to make the code more readable - // Find monotonicity violating sequence, if any. - while (j < len - 1 && input(j)._1 > input(j + 1)._1) { - j = j + 1 - } + // blockStart and blockEnd have identical implementations. We create two different + // functions to make the code more expressive + def blockEnd(start: Int): Int = blockBounds(start) + def blockStart(end: Int): Int = blockBounds(end) - // If monotonicity was not violated, move to next data point. - if (i == j) { - i = i + 1 - } else { - // Otherwise pool the violating sequence - // and check if pooling caused monotonicity violation in previously processed points. - while (i >= 0 && input(i)._1 > input(i + 1)._1) { - pool(input, i, j) - i = i - 1 - } + // the next block starts at the index after the end of this block + def nextBlock(start: Int): Int = blockEnd(start) + 1 - i = j - } + // the previous block ends at the index before the start of this block + // we then use blockStart to find the start + def prevBlock(start: Int): Int = blockStart(start - 1) + + // Merge two adjacent blocks, updating blockBounds and weights to reflect the merge + // Return the start index of the merged block + def merge(block1: Int, block2: Int): Int = { + assert( + blockEnd(block1) + 1 == block2, + s"Attempting to merge non-consecutive blocks [${block1}, ${blockEnd(block1)}]" + + s" and [${block2}, ${blockEnd(block2)}]. This is likely a bug in the isotonic regression" + + " implementation. Please file a bug report." + ) + blockBounds(block1) = blockEnd(block2) + blockBounds(blockEnd(block2)) = block1 + val w1 = weights(block1) + val w2 = weights(block2) + weights(block1) = (w1._1 + w2._1, w1._2 + w2._2) + block1 } - // For points having the same prediction, we only keep two boundary points. - val compressed = ArrayBuffer.empty[(Double, Double, Double)] + // average value of a block + def average(start: Int): Double = weights(start)._2 / weights(start)._1 - var (curLabel, curFeature, curWeight) = input.head - var rightBound = curFeature - def merge(): Unit = { - compressed += ((curLabel, curFeature, curWeight)) - if (rightBound > curFeature) { - compressed += ((curLabel, rightBound, 0.0)) + // Implement Algorithm PAV from [3]. + // Merge on >= instead of > because it eliminates adjacent blocks with the same average, and we + // want to compress our output as much as possible. Both give correct results. + var i = 0 + while (nextBlock(i) < cleanInput.length) { + if (average(i) >= average(nextBlock(i))) { + merge(i, nextBlock(i)) + while((i > 0) && (average(prevBlock(i)) >= average(i))) { + i = merge(prevBlock(i), i) + } + } else { + i = nextBlock(i) } } - i = 1 - while (i < input.length) { - val (label, feature, weight) = input(i) - if (label == curLabel) { - curWeight += weight - rightBound = feature + + // construct the output by walking through the blocks in order + val output = ArrayBuffer.empty[(Double, Double, Double)] + i = 0 + while (i < cleanInput.length) { + // If block size is > 1, a point at the start and end of the block, + // each receiving half the weight. Otherwise, a single point with + // all the weight. + if (cleanInput(blockEnd(i))._2 > cleanInput(i)._2) { + output += ((average(i), cleanInput(i)._2, weights(i)._1 / 2)) + output += ((average(i), cleanInput(blockEnd(i))._2, weights(i)._1 / 2)) } else { - merge() - curLabel = label - curFeature = feature - curWeight = weight - rightBound = curFeature + output += ((average(i), cleanInput(i)._2, weights(i)._1)) } - i += 1 + i = nextBlock(i) } - merge() - compressed.toArray + output.toArray } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 499c80767aea7..e5aece779826d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -83,8 +83,8 @@ object DecisionTree extends Serializable with Logging { * categorical), depth of the tree, quantile calculation strategy, etc. * @return DecisionTreeModel that can be used for prediction. * - * @note Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] - * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` + * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` * is recommended to clearly separate classification and regression. */ @Since("1.0.0") @@ -105,8 +105,8 @@ object DecisionTree extends Serializable with Logging { * 1 internal node + 2 leaf nodes). * @return DecisionTreeModel that can be used for prediction. * - * @note Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] - * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` + * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` * is recommended to clearly separate classification and regression. */ @Since("1.0.0") @@ -133,8 +133,8 @@ object DecisionTree extends Serializable with Logging { * @param numClasses Number of classes for classification. Default value of 2. * @return DecisionTreeModel that can be used for prediction. * - * @note Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] - * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` + * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` * is recommended to clearly separate classification and regression. */ @Since("1.2.0") @@ -167,8 +167,8 @@ object DecisionTree extends Serializable with Logging { * indexed from 0: {0, 1, ..., k-1}. * @return DecisionTreeModel that can be used for prediction. * - * @note Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] - * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` + * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` * is recommended to clearly separate classification and regression. */ @Since("1.0.0") @@ -218,7 +218,7 @@ object DecisionTree extends Serializable with Logging { } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * Java-friendly API for `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` */ @Since("1.1.0") def trainClassifier( @@ -262,7 +262,7 @@ object DecisionTree extends Serializable with Logging { } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * Java-friendly API for `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` */ @Since("1.1.0") def trainRegressor( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index 3e85678906b33..df2c1b02f4f40 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -136,7 +136,7 @@ object GradientBoostedTrees extends Logging { } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoostedTrees$#train]] + * Java-friendly API for `org.apache.spark.mllib.tree.GradientBoostedTrees.train` */ @Since("1.2.0") def train( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index 1f6cb086cefa9..d1331a57de27b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -172,7 +172,7 @@ object RandomForest extends Serializable with Logging { } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]] + * Java-friendly API for `org.apache.spark.mllib.tree.RandomForest.trainClassifier` */ @Since("1.2.0") def trainClassifier( @@ -259,7 +259,7 @@ object RandomForest extends Serializable with Logging { } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]] + * Java-friendly API for `org.apache.spark.mllib.tree.RandomForest.trainRegressor` */ @Since("1.2.0") def trainRegressor( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index 5d92ce495b04d..9339f0a23c1bd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -20,7 +20,6 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.util.MLUtils - /** * :: DeveloperApi :: * Class for log loss calculation (for classification). @@ -32,7 +31,7 @@ import org.apache.spark.mllib.util.MLUtils */ @Since("1.2.0") @DeveloperApi -object LogLoss extends Loss { +object LogLoss extends ClassificationLoss { /** * Method to calculate the loss gradients for the gradient boosting calculation for binary @@ -52,4 +51,11 @@ object LogLoss extends Loss { // The following is equivalent to 2.0 * log(1 + exp(-margin)) but more numerically stable. 2.0 * MLUtils.log1pExp(-margin) } + + /** + * Returns the estimated probability of a label of 1.0. + */ + override private[spark] def computeProbability(margin: Double): Double = { + 1.0 / (1.0 + math.exp(-2.0 * margin)) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index 09274a2e1b2ac..e7ffb3f8f53c1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -22,7 +22,6 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.rdd.RDD - /** * :: DeveloperApi :: * Trait for adding "pluggable" loss functions for the gradient boosting algorithm. @@ -67,3 +66,10 @@ trait Loss extends Serializable { */ private[spark] def computeError(prediction: Double, label: Double): Double } + +private[spark] trait ClassificationLoss extends Loss { + /** + * Computes the class probability given the margin. + */ + private[spark] def computeProbability(margin: Double): Double +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 6bb3271aacb44..95f904dac552c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -34,7 +34,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.BernoulliCellSampler /** - * Helper methods to load, save and pre-process data used in ML Lib. + * Helper methods to load, save and pre-process data used in MLLib. */ @Since("0.8.0") object MLUtils extends Logging { @@ -149,7 +149,7 @@ object MLUtils extends Logging { * Save labeled data in LIBSVM format. * @param data an RDD of LabeledPoint to be saved * @param dir directory to save the data - * @see [[org.apache.spark.mllib.util.MLUtils#loadLibSVMFile]] + * @see `org.apache.spark.mllib.util.MLUtils.loadLibSVMFile` */ @Since("1.0.0") def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) { diff --git a/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala index 03e0c536a973e..ec45e32d412a9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml import org.apache.spark.SparkFunSuite import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.shared.HasWeightCol import org.apache.spark.ml.util._ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Dataset @@ -30,24 +31,28 @@ class PredictorSuite extends SparkFunSuite with MLlibTestSparkContext { import PredictorSuite._ - test("should support all NumericType labels and not support other types") { + test("should support all NumericType labels and weights, and not support other types") { val df = spark.createDataFrame(Seq( - (0, Vectors.dense(0, 2, 3)), - (1, Vectors.dense(0, 3, 9)), - (0, Vectors.dense(0, 2, 6)) - )).toDF("label", "features") + (0, 1, Vectors.dense(0, 2, 3)), + (1, 2, Vectors.dense(0, 3, 9)), + (0, 3, Vectors.dense(0, 2, 6)) + )).toDF("label", "weight", "features") val types = Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) - val predictor = new MockPredictor() + val predictor = new MockPredictor().setWeightCol("weight") types.foreach { t => - predictor.fit(df.select(col("label").cast(t), col("features"))) + predictor.fit(df.select(col("label").cast(t), col("weight").cast(t), col("features"))) } intercept[IllegalArgumentException] { - predictor.fit(df.select(col("label").cast(StringType), col("features"))) + predictor.fit(df.select(col("label").cast(StringType), col("weight"), col("features"))) + } + + intercept[IllegalArgumentException] { + predictor.fit(df.select(col("label"), col("weight").cast(StringType), col("features"))) } } } @@ -55,12 +60,15 @@ class PredictorSuite extends SparkFunSuite with MLlibTestSparkContext { object PredictorSuite { class MockPredictor(override val uid: String) - extends Predictor[Vector, MockPredictor, MockPredictionModel] { + extends Predictor[Vector, MockPredictor, MockPredictionModel] with HasWeightCol { def this() = this(Identifiable.randomUID("mockpredictor")) + def setWeightCol(value: String): this.type = set(weightCol, value) + override def train(dataset: Dataset[_]): MockPredictionModel = { require(dataset.schema("label").dataType == DoubleType) + require(dataset.schema("weight").dataType == DoubleType) new MockPredictionModel(uid) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index 7c36745ab213b..0598943c3d4be 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -17,20 +17,24 @@ package org.apache.spark.ml.classification +import com.github.fommil.netlib.BLAS + import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.feature.LabeledPoint -import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.tree.loss.LogLoss import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.util.Utils /** @@ -49,6 +53,8 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext private var data: RDD[LabeledPoint] = _ private var trainData: RDD[LabeledPoint] = _ private var validationData: RDD[LabeledPoint] = _ + private val eps: Double = 1e-5 + private val absEps: Double = 1e-8 override def beforeAll() { super.beforeAll() @@ -66,10 +72,156 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext ParamsSuite.checkParams(new GBTClassifier) val model = new GBTClassificationModel("gbtc", Array(new DecisionTreeRegressionModel("dtr", new LeafNode(0.0, 0.0, null), 1)), - Array(1.0), 1) + Array(1.0), 1, 2) ParamsSuite.checkParams(model) } + test("GBTClassifier: default params") { + val gbt = new GBTClassifier + assert(gbt.getLabelCol === "label") + assert(gbt.getFeaturesCol === "features") + assert(gbt.getPredictionCol === "prediction") + assert(gbt.getRawPredictionCol === "rawPrediction") + assert(gbt.getProbabilityCol === "probability") + val df = trainData.toDF() + val model = gbt.fit(df) + model.transform(df) + .select("label", "probability", "prediction", "rawPrediction") + .collect() + intercept[NoSuchElementException] { + model.getThresholds + } + assert(model.getFeaturesCol === "features") + assert(model.getPredictionCol === "prediction") + assert(model.getRawPredictionCol === "rawPrediction") + assert(model.getProbabilityCol === "probability") + assert(model.hasParent) + + // copied model must have the same parent. + MLTestingUtils.checkCopy(model) + } + + test("setThreshold, getThreshold") { + val gbt = new GBTClassifier + + // default + withClue("GBTClassifier should not have thresholds set by default.") { + intercept[NoSuchElementException] { + gbt.getThresholds + } + } + + // Set via thresholds + val gbt2 = new GBTClassifier + val threshold = Array(0.3, 0.7) + gbt2.setThresholds(threshold) + assert(gbt2.getThresholds === threshold) + } + + test("thresholds prediction") { + val gbt = new GBTClassifier + val df = trainData.toDF() + val binaryModel = gbt.fit(df) + + // should predict all zeros + binaryModel.setThresholds(Array(0.0, 1.0)) + val binaryZeroPredictions = binaryModel.transform(df).select("prediction").collect() + assert(binaryZeroPredictions.forall(_.getDouble(0) === 0.0)) + + // should predict all ones + binaryModel.setThresholds(Array(1.0, 0.0)) + val binaryOnePredictions = binaryModel.transform(df).select("prediction").collect() + assert(binaryOnePredictions.forall(_.getDouble(0) === 1.0)) + + + val gbtBase = new GBTClassifier + val model = gbtBase.fit(df) + val basePredictions = model.transform(df).select("prediction").collect() + + // constant threshold scaling is the same as no thresholds + binaryModel.setThresholds(Array(1.0, 1.0)) + val scaledPredictions = binaryModel.transform(df).select("prediction").collect() + assert(scaledPredictions.zip(basePredictions).forall { case (scaled, base) => + scaled.getDouble(0) === base.getDouble(0) + }) + + // force it to use the predict method + model.setRawPredictionCol("").setProbabilityCol("").setThresholds(Array(0, 1)) + val predictionsWithPredict = model.transform(df).select("prediction").collect() + assert(predictionsWithPredict.forall(_.getDouble(0) === 0.0)) + } + + test("GBTClassifier: Predictor, Classifier methods") { + val rawPredictionCol = "rawPrediction" + val predictionCol = "prediction" + val labelCol = "label" + val featuresCol = "features" + val probabilityCol = "probability" + + val gbt = new GBTClassifier().setSeed(123) + val trainingDataset = trainData.toDF(labelCol, featuresCol) + val gbtModel = gbt.fit(trainingDataset) + assert(gbtModel.numClasses === 2) + val numFeatures = trainingDataset.select(featuresCol).first().getAs[Vector](0).size + assert(gbtModel.numFeatures === numFeatures) + + val blas = BLAS.getInstance() + + val validationDataset = validationData.toDF(labelCol, featuresCol) + val results = gbtModel.transform(validationDataset) + // check that raw prediction is tree predictions dot tree weights + results.select(rawPredictionCol, featuresCol).collect().foreach { + case Row(raw: Vector, features: Vector) => + assert(raw.size === 2) + val treePredictions = gbtModel.trees.map(_.rootNode.predictImpl(features).prediction) + val prediction = blas.ddot(gbtModel.numTrees, treePredictions, 1, gbtModel.treeWeights, 1) + assert(raw ~== Vectors.dense(-prediction, prediction) relTol eps) + } + + // Compare rawPrediction with probability + results.select(rawPredictionCol, probabilityCol).collect().foreach { + case Row(raw: Vector, prob: Vector) => + assert(raw.size === 2) + assert(prob.size === 2) + // Note: we should check other loss types for classification if they are added + val predFromRaw = raw.toDense.values.map(value => LogLoss.computeProbability(value)) + assert(prob(0) ~== predFromRaw(0) relTol eps) + assert(prob(1) ~== predFromRaw(1) relTol eps) + assert(prob(0) + prob(1) ~== 1.0 absTol absEps) + } + + // Compare prediction with probability + results.select(predictionCol, probabilityCol).collect().foreach { + case Row(pred: Double, prob: Vector) => + val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 + assert(pred == predFromProb) + } + + // force it to use raw2prediction + gbtModel.setRawPredictionCol(rawPredictionCol).setProbabilityCol("") + val resultsUsingRaw2Predict = + gbtModel.transform(validationDataset).select(predictionCol).as[Double].collect() + resultsUsingRaw2Predict.zip(results.select(predictionCol).as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + // force it to use probability2prediction + gbtModel.setRawPredictionCol("").setProbabilityCol(probabilityCol) + val resultsUsingProb2Predict = + gbtModel.transform(validationDataset).select(predictionCol).as[Double].collect() + resultsUsingProb2Predict.zip(results.select(predictionCol).as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + // force it to use predict + gbtModel.setRawPredictionCol("").setProbabilityCol("") + val resultsUsingPredict = + gbtModel.transform(validationDataset).select(predictionCol).as[Double].collect() + resultsUsingPredict.zip(results.select(predictionCol).as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + } + test("GBT parameter stepSize should be in interval (0, 1]") { withClue("GBT parameter stepSize should be in interval (0, 1]") { intercept[IllegalArgumentException] { @@ -246,7 +398,8 @@ private object GBTClassifierSuite extends SparkFunSuite { val newModel = gbt.fit(newData) // Use parent from newTree since this is not checked anyways. val oldModelAsNew = GBTClassificationModel.fromOld( - oldModel, newModel.parent.asInstanceOf[GBTClassifier], categoricalFeatures, numFeatures) + oldModel, newModel.parent.asInstanceOf[GBTClassifier], categoricalFeatures, + numFeatures, numClasses = 2) TreeTests.checkEqual(oldModelAsNew, newModel) assert(newModel.numFeatures === numFeatures) assert(oldModelAsNew.numFeatures === numFeatures) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala new file mode 100644 index 0000000000000..ee2aefee7a6db --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification + +import scala.util.Random + +import breeze.linalg.{DenseVector => BDV} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LinearSVCSuite._ +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{Dataset, Row} + + +class LinearSVCSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + private val nPoints = 50 + @transient var smallBinaryDataset: Dataset[_] = _ + @transient var smallValidationDataset: Dataset[_] = _ + @transient var binaryDataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 + val B = -1.5 + val C = 1.0 + smallBinaryDataset = generateSVMInput(A, Array[Double](B, C), nPoints, 42).toDF() + smallValidationDataset = generateSVMInput(A, Array[Double](B, C), nPoints, 17).toDF() + binaryDataset = generateSVMInput(1.0, Array[Double](1.0, 2.0, 3.0, 4.0), 10000, 42).toDF() + } + + /** + * Enable the ignored test to export the dataset into CSV format, + * so we can validate the training accuracy compared with R's e1071 package. + */ + ignore("export test data into CSV format") { + binaryDataset.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile("target/tmp/LinearSVC/binaryDataset") + } + + test("Linear SVC binary classification") { + val svm = new LinearSVC() + val model = svm.fit(smallBinaryDataset) + assert(model.transform(smallValidationDataset) + .where("prediction=label").count() > nPoints * 0.8) + } + + test("Linear SVC binary classification with regularization") { + val svm = new LinearSVC() + val model = svm.setRegParam(0.1).fit(smallBinaryDataset) + assert(model.transform(smallValidationDataset) + .where("prediction=label").count() > nPoints * 0.8) + } + + test("params") { + ParamsSuite.checkParams(new LinearSVC) + val model = new LinearSVCModel("linearSVC", Vectors.dense(0.0), 0.0) + ParamsSuite.checkParams(model) + } + + test("linear svc: default params") { + val lsvc = new LinearSVC() + assert(lsvc.getRegParam === 0.0) + assert(lsvc.getMaxIter === 100) + assert(lsvc.getFitIntercept) + assert(lsvc.getTol === 1E-6) + assert(lsvc.getStandardization) + assert(!lsvc.isDefined(lsvc.weightCol)) + assert(lsvc.getThreshold === 0.0) + assert(lsvc.getAggregationDepth === 2) + assert(lsvc.getLabelCol === "label") + assert(lsvc.getFeaturesCol === "features") + assert(lsvc.getPredictionCol === "prediction") + assert(lsvc.getRawPredictionCol === "rawPrediction") + val model = lsvc.setMaxIter(5).fit(smallBinaryDataset) + model.transform(smallBinaryDataset) + .select("label", "prediction", "rawPrediction") + .collect() + assert(model.getThreshold === 0.0) + assert(model.getFeaturesCol === "features") + assert(model.getPredictionCol === "prediction") + assert(model.getRawPredictionCol === "rawPrediction") + assert(model.intercept !== 0.0) + assert(model.hasParent) + assert(model.numFeatures === 2) + + // copied model must have the same parent. + MLTestingUtils.checkCopy(model) + } + + test("linear svc doesn't fit intercept when fitIntercept is off") { + val lsvc = new LinearSVC().setFitIntercept(false).setMaxIter(5) + val model = lsvc.fit(smallBinaryDataset) + assert(model.intercept === 0.0) + + val lsvc2 = new LinearSVC().setFitIntercept(true).setMaxIter(5) + val model2 = lsvc2.fit(smallBinaryDataset) + assert(model2.intercept !== 0.0) + } + + test("linearSVC with sample weights") { + def modelEquals(m1: LinearSVCModel, m2: LinearSVCModel): Unit = { + assert(m1.coefficients ~== m2.coefficients absTol 0.05) + assert(m1.intercept ~== m2.intercept absTol 0.05) + } + + val estimator = new LinearSVC().setRegParam(0.01).setTol(0.01) + val dataset = smallBinaryDataset + MLTestingUtils.testArbitrarilyScaledWeights[LinearSVCModel, LinearSVC]( + dataset.as[LabeledPoint], estimator, modelEquals) + MLTestingUtils.testOutliersWithSmallWeights[LinearSVCModel, LinearSVC]( + dataset.as[LabeledPoint], estimator, 2, modelEquals) + MLTestingUtils.testOversamplingVsWeighting[LinearSVCModel, LinearSVC]( + dataset.as[LabeledPoint], estimator, modelEquals, 42L) + } + + test("linearSVC comparison with R e1071 and scikit-learn") { + val trainer1 = new LinearSVC() + .setRegParam(0.00002) // set regParam = 2.0 / datasize / c + .setMaxIter(200) + .setTol(1e-4) + val model1 = trainer1.fit(binaryDataset) + + /* + Use the following R code to load the data and train the model using glmnet package. + + library(e1071) + data <- read.csv("path/target/tmp/LinearSVC/binaryDataset/part-00000", header=FALSE) + label <- factor(data$V1) + features <- as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + svm_model <- svm(features, label, type='C', kernel='linear', cost=10, scale=F, tolerance=1e-4) + w <- -t(svm_model$coefs) %*% svm_model$SV + w + svm_model$rho + + > w + data.V2 data.V3 data.V4 data.V5 + [1,] 7.310338 14.89741 22.21005 29.83508 + > svm_model$rho + [1] 7.440177 + + */ + val coefficientsR = Vectors.dense(7.310338, 14.89741, 22.21005, 29.83508) + val interceptR = 7.440177 + assert(model1.intercept ~== interceptR relTol 1E-2) + assert(model1.coefficients ~== coefficientsR relTol 1E-2) + + /* + Use the following python code to load the data and train the model using scikit-learn package. + + import numpy as np + from sklearn import svm + f = open("path/target/tmp/LinearSVC/binaryDataset/part-00000") + data = np.loadtxt(f, delimiter=",") + X = data[:, 1:] # select columns 1 through end + y = data[:, 0] # select column 0 as label + clf = svm.LinearSVC(fit_intercept=True, C=10, loss='hinge', tol=1e-4, random_state=42) + m = clf.fit(X, y) + print m.coef_ + print m.intercept_ + + [[ 7.24690165 14.77029087 21.99924004 29.5575729 ]] + [ 7.36947518] + */ + + val coefficientsSK = Vectors.dense(7.24690165, 14.77029087, 21.99924004, 29.5575729) + val interceptSK = 7.36947518 + assert(model1.intercept ~== interceptSK relTol 1E-3) + assert(model1.coefficients ~== coefficientsSK relTol 4E-3) + } + + test("read/write: SVM") { + def checkModelData(model: LinearSVCModel, model2: LinearSVCModel): Unit = { + assert(model.intercept === model2.intercept) + assert(model.coefficients === model2.coefficients) + assert(model.numFeatures === model2.numFeatures) + } + val svm = new LinearSVC() + testEstimatorAndModelReadWrite(svm, smallBinaryDataset, LinearSVCSuite.allParamSettings, + checkModelData) + } +} + +object LinearSVCSuite { + + val allParamSettings: Map[String, Any] = Map( + "regParam" -> 0.01, + "maxIter" -> 2, // intentionally small + "fitIntercept" -> true, + "tol" -> 0.8, + "standardization" -> false, + "threshold" -> 0.6, + "predictionCol" -> "myPredict", + "rawPredictionCol" -> "myRawPredict", + "aggregationDepth" -> 3 + ) + + // Generate noisy input of the form Y = signum(x.dot(weights) + intercept + noise) + def generateSVMInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val weightsMat = new BDV(weights) + val x = Array.fill[Array[Double]](nPoints)( + Array.fill[Double](weights.length)(rnd.nextDouble() * 2.0 - 1.0)) + val y = x.map { xi => + val yD = new BDV(xi).dot(weightsMat) + intercept + 0.01 * rnd.nextGaussian() + if (yD > 0) 1.0 else 0.0 + } + y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) + } + +} + diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index c14dcbd55287d..43547a4aafcb9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -2066,7 +2066,7 @@ class LogisticRegressionSuite checkModelData) } - test("should support all NumericType labels and not support other types") { + test("should support all NumericType labels and weights, and not support other types") { val lr = new LogisticRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[LogisticRegressionModel, LogisticRegression]( lr, spark) { (expected, actual) => diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 2a69ef1c3ed01..37d7991fe8dd8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -283,7 +283,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa testEstimatorAndModelReadWrite(nb, dataset, NaiveBayesSuite.allParamSettings, checkModelData) } - test("should support all NumericType labels and not support other types") { + test("should support all NumericType labels and weights, and not support other types") { val nb = new NaiveBayes() MLTestingUtils.checkNumericTypes[NaiveBayesModel, NaiveBayes]( nb, spark) { (expected, actual) => @@ -324,8 +324,8 @@ object NaiveBayesSuite { sample: Int = 10): Seq[LabeledPoint] = { val D = theta(0).length val rnd = new Random(seed) - val _pi = pi.map(math.pow(math.E, _)) - val _theta = theta.map(row => row.map(math.pow(math.E, _))) + val _pi = pi.map(math.exp) + val _theta = theta.map(row => row.map(math.exp)) for (i <- 0 until nPoints) yield { val y = calcLabel(rnd.nextDouble(), _pi) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index fc491cd6161fd..30513c1e276ae 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -29,9 +29,12 @@ class BisectingKMeansSuite final val k = 5 @transient var dataset: Dataset[_] = _ + @transient var sparseDataset: Dataset[_] = _ + override def beforeAll(): Unit = { super.beforeAll() dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k) + sparseDataset = KMeansSuite.generateSparseData(spark, 10, 1000, 42) } test("default parameters") { @@ -51,6 +54,22 @@ class BisectingKMeansSuite assert(copiedModel.hasSummary) } + test("SPARK-16473: Verify Bisecting K-Means does not fail in edge case where" + + "one cluster is empty after split") { + val bkm = new BisectingKMeans() + .setK(k) + .setMinDivisibleClusterSize(4) + .setMaxIter(4) + .setSeed(123) + + // Verify fit does not fail on very sparse data + val model = bkm.fit(sparseDataset) + val result = model.transform(sparseDataset) + val numClusters = result.select("prediction").distinct().collect().length + // Verify we hit the edge case + assert(numClusters < k && numClusters > 1) + } + test("setter/getter") { val bkm = new BisectingKMeans() .setK(9) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala index a362aeea3962b..c500c5b3e365a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala @@ -53,6 +53,20 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext rDataset = rData.map(FeatureData).toDF() } + test("gmm fails on high dimensional data") { + val df = Seq( + Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(0, 4), Array(3.0, 8.0)), + Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(1, 5), Array(4.0, 9.0))) + .map(Tuple1.apply).toDF("features") + val gm = new GaussianMixture() + withClue(s"GMM should restrict the maximum number of features to be < " + + s"${GaussianMixture.MAX_NUM_FEATURES}") { + intercept[IllegalArgumentException] { + gm.fit(df) + } + } + } + test("default parameters") { val gm = new GaussianMixture() @@ -207,6 +221,10 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext [,1] [,2] [1,] 0.2961543 0.160783 [2,] 0.1607830 1.008878 + + model$loglik + + [1] -46.89499 */ val weights = Array(0.5333333, 0.4666667) val means = Array(Vectors.dense(10.363673, 9.897081), Vectors.dense(0.11731091, -0.06192351)) @@ -219,6 +237,9 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext val expected = new GaussianMixtureModel("dummy", weights, gaussians) val actual = new GaussianMixture().setK(2).setSeed(seed).fit(rDataset) modelEquals(expected, actual) + + val llk = actual.summary.logLikelihood + assert(llk ~== -46.89499 absTol 1E-5) } test("upper triangular matrix unpacking") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index c1b7242e11a8f..e10127f7d108f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.clustering +import scala.util.Random + import org.apache.spark.SparkFunSuite import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap @@ -160,6 +162,17 @@ object KMeansSuite { spark.createDataFrame(rdd) } + def generateSparseData(spark: SparkSession, rows: Int, dim: Int, seed: Int): DataFrame = { + val sc = spark.sparkContext + val random = new Random(seed) + val nnz = random.nextInt(dim) + val rdd = sc.parallelize(1 to rows) + .map(i => Vectors.sparse(dim, random.shuffle(0 to dim - 1).slice(0, nnz).sorted.toArray, + Array.fill(nnz)(random.nextDouble()))) + .map(v => new TestRow(v)) + spark.createDataFrame(rdd) + } + /** * Mapping from all Params to valid settings which differ from the defaults. * This is useful for tests which need to exercise all Params, such as save/load. diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index ed24c1e16a130..ea059858a58b7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -191,6 +191,8 @@ class GeneralizedLinearRegressionSuite assert(!glr.isDefined(glr.weightCol)) assert(glr.getRegParam === 0.0) assert(glr.getSolver == "irls") + assert(glr.getVariancePower === 0.0) + // TODO: Construct model directly instead of via fitting. val model = glr.setFamily("gaussian").setLink("identity") .fit(datasetGaussianIdentity) @@ -266,7 +268,7 @@ class GeneralizedLinearRegressionSuite assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with gaussian family, " + s"$link link and fitIntercept = $fitIntercept.") - val familyLink = new FamilyAndLink(Gaussian, Link.fromName(link)) + val familyLink = FamilyAndLink(trainer) model.transform(dataset).select("features", "prediction", "linkPrediction").collect() .foreach { case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => @@ -382,7 +384,7 @@ class GeneralizedLinearRegressionSuite assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with binomial family, " + s"$link link and fitIntercept = $fitIntercept.") - val familyLink = new FamilyAndLink(Binomial, Link.fromName(link)) + val familyLink = FamilyAndLink(trainer) model.transform(dataset).select("features", "prediction", "linkPrediction").collect() .foreach { case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => @@ -454,7 +456,7 @@ class GeneralizedLinearRegressionSuite assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with poisson family, " + s"$link link and fitIntercept = $fitIntercept.") - val familyLink = new FamilyAndLink(Poisson, Link.fromName(link)) + val familyLink = FamilyAndLink(trainer) model.transform(dataset).select("features", "prediction", "linkPrediction").collect() .foreach { case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => @@ -553,14 +555,14 @@ class GeneralizedLinearRegressionSuite for ((link, dataset) <- Seq(("inverse", datasetGammaInverse), ("identity", datasetGammaIdentity), ("log", datasetGammaLog))) { for (fitIntercept <- Seq(false, true)) { - val trainer = new GeneralizedLinearRegression().setFamily("gamma").setLink(link) + val trainer = new GeneralizedLinearRegression().setFamily("Gamma").setLink(link) .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") val model = trainer.fit(dataset) val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with gamma family, " + s"$link link and fitIntercept = $fitIntercept.") - val familyLink = new FamilyAndLink(Gamma, Link.fromName(link)) + val familyLink = FamilyAndLink(trainer) model.transform(dataset).select("features", "prediction", "linkPrediction").collect() .foreach { case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => @@ -578,6 +580,169 @@ class GeneralizedLinearRegressionSuite } } + test("generalized linear regression: tweedie family against glm") { + /* + R code: + library(statmod) + df <- as.data.frame(matrix(c( + 1.0, 1.0, 0.0, 5.0, + 0.5, 1.0, 1.0, 2.0, + 1.0, 1.0, 2.0, 1.0, + 2.0, 1.0, 3.0, 3.0), 4, 4, byrow = TRUE)) + + f1 <- V1 ~ -1 + V3 + V4 + f2 <- V1 ~ V3 + V4 + + for (f in c(f1, f2)) { + for (lp in c(0, 1, -1)) + for (vp in c(1.6, 2.5)) { + model <- glm(f, df, family = tweedie(var.power = vp, link.power = lp)) + print(as.vector(coef(model))) + } + } + [1] 0.1496480 -0.0122283 + [1] 0.1373567 -0.0120673 + [1] 0.3919109 0.1846094 + [1] 0.3684426 0.1810662 + [1] 0.1759887 0.2195818 + [1] 0.1108561 0.2059430 + [1] -1.3163732 0.4378139 0.2464114 + [1] -1.4396020 0.4817364 0.2680088 + [1] -0.7090230 0.6256309 0.3294324 + [1] -0.9524928 0.7304267 0.3792687 + [1] 2.1188978 -0.3360519 -0.2067023 + [1] 2.1659028 -0.3499170 -0.2128286 + */ + val datasetTweedie = Seq( + Instance(1.0, 1.0, Vectors.dense(0.0, 5.0)), + Instance(0.5, 1.0, Vectors.dense(1.0, 2.0)), + Instance(1.0, 1.0, Vectors.dense(2.0, 1.0)), + Instance(2.0, 1.0, Vectors.dense(3.0, 3.0)) + ).toDF() + + val expected = Seq( + Vectors.dense(0, 0.149648, -0.0122283), + Vectors.dense(0, 0.1373567, -0.0120673), + Vectors.dense(0, 0.3919109, 0.1846094), + Vectors.dense(0, 0.3684426, 0.1810662), + Vectors.dense(0, 0.1759887, 0.2195818), + Vectors.dense(0, 0.1108561, 0.205943), + Vectors.dense(-1.3163732, 0.4378139, 0.2464114), + Vectors.dense(-1.439602, 0.4817364, 0.2680088), + Vectors.dense(-0.709023, 0.6256309, 0.3294324), + Vectors.dense(-0.9524928, 0.7304267, 0.3792687), + Vectors.dense(2.1188978, -0.3360519, -0.2067023), + Vectors.dense(2.1659028, -0.349917, -0.2128286)) + + import GeneralizedLinearRegression._ + + var idx = 0 + for (fitIntercept <- Seq(false, true); + linkPower <- Seq(0.0, 1.0, -1.0); + variancePower <- Seq(1.6, 2.5)) { + val trainer = new GeneralizedLinearRegression().setFamily("tweedie") + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + .setVariancePower(variancePower).setLinkPower(linkPower) + val model = trainer.fit(datasetTweedie) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with tweedie family, " + + s"linkPower = $linkPower, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + + val familyLink = FamilyAndLink(trainer) + model.transform(datasetTweedie).select("features", "prediction", "linkPrediction").collect() + .foreach { + case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"tweedie family, linkPower = $linkPower, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with tweedie family, linkPower = $linkPower, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + } + idx += 1 + } + } + + test("generalized linear regression: tweedie family against glm (default power link)") { + /* + R code: + library(statmod) + df <- as.data.frame(matrix(c( + 1.0, 1.0, 0.0, 5.0, + 0.5, 1.0, 1.0, 2.0, + 1.0, 1.0, 2.0, 1.0, + 2.0, 1.0, 3.0, 3.0), 4, 4, byrow = TRUE)) + var.power <- c(0, 1, 2, 1.5) + f1 <- V1 ~ -1 + V3 + V4 + f2 <- V1 ~ V3 + V4 + for (f in c(f1, f2)) { + for (vp in var.power) { + model <- glm(f, df, family = tweedie(var.power = vp)) + print(as.vector(coef(model))) + } + } + [1] 0.4310345 0.1896552 + [1] 0.15776482 -0.01189032 + [1] 0.1468853 0.2116519 + [1] 0.2282601 0.2132775 + [1] -0.5158730 0.5555556 0.2936508 + [1] -1.2689559 0.4230934 0.2388465 + [1] 2.137852 -0.341431 -0.209090 + [1] 1.5953393 -0.1884985 -0.1106335 + */ + val datasetTweedie = Seq( + Instance(1.0, 1.0, Vectors.dense(0.0, 5.0)), + Instance(0.5, 1.0, Vectors.dense(1.0, 2.0)), + Instance(1.0, 1.0, Vectors.dense(2.0, 1.0)), + Instance(2.0, 1.0, Vectors.dense(3.0, 3.0)) + ).toDF() + + val expected = Seq( + Vectors.dense(0, 0.4310345, 0.1896552), + Vectors.dense(0, 0.15776482, -0.01189032), + Vectors.dense(0, 0.1468853, 0.2116519), + Vectors.dense(0, 0.2282601, 0.2132775), + Vectors.dense(-0.515873, 0.5555556, 0.2936508), + Vectors.dense(-1.2689559, 0.4230934, 0.2388465), + Vectors.dense(2.137852, -0.341431, -0.20909), + Vectors.dense(1.5953393, -0.1884985, -0.1106335)) + + import GeneralizedLinearRegression._ + + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + for (variancePower <- Seq(0.0, 1.0, 2.0, 1.5)) { + val trainer = new GeneralizedLinearRegression().setFamily("tweedie") + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + .setVariancePower(variancePower) + val model = trainer.fit(datasetTweedie) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with tweedie family, " + + s"fitIntercept = $fitIntercept and variancePower = $variancePower.") + + val familyLink = FamilyAndLink(trainer) + model.transform(datasetTweedie).select("features", "prediction", "linkPrediction").collect() + .foreach { + case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"tweedie family, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with tweedie family, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + } + idx += 1 + } + } + } + test("glm summary: gaussian family with weight") { /* R code: @@ -758,7 +923,7 @@ class GeneralizedLinearRegressionSuite 0.028480 0.069123 0.935495 -0.049613 */ val trainer = new GeneralizedLinearRegression() - .setFamily("binomial") + .setFamily("Binomial") .setWeightCol("weight") .setFitIntercept(false) @@ -875,7 +1040,7 @@ class GeneralizedLinearRegressionSuite -0.4378554 0.2189277 0.1459518 -0.1094638 */ val trainer = new GeneralizedLinearRegression() - .setFamily("poisson") + .setFamily("Poisson") .setWeightCol("weight") .setFitIntercept(true) @@ -990,7 +1155,7 @@ class GeneralizedLinearRegressionSuite -0.6344390 0.3172195 0.2114797 -0.1586097 */ val trainer = new GeneralizedLinearRegression() - .setFamily("gamma") + .setFamily("Gamma") .setWeightCol("weight") val model = trainer.fit(datasetWithWeight) @@ -1052,6 +1217,121 @@ class GeneralizedLinearRegressionSuite assert(summary.solver === "irls") } + test("glm summary: tweedie family with weight") { + /* + R code: + + library(statmod) + df <- as.data.frame(matrix(c( + 1.0, 1.0, 0.0, 5.0, + 0.5, 2.0, 1.0, 2.0, + 1.0, 3.0, 2.0, 1.0, + 0.0, 4.0, 3.0, 3.0), 4, 4, byrow = TRUE)) + + model <- glm(V1 ~ -1 + V3 + V4, data = df, weights = V2, + family = tweedie(var.power = 1.6, link.power = 0)) + summary(model) + + Deviance Residuals: + 1 2 3 4 + 0.6210 -0.0515 1.6935 -3.2539 + + Coefficients: + Estimate Std. Error t value Pr(>|t|) + V3 -0.4087 0.5205 -0.785 0.515 + V4 -0.1212 0.4082 -0.297 0.794 + + (Dispersion parameter for Tweedie family taken to be 3.830036) + + Null deviance: 20.702 on 4 degrees of freedom + Residual deviance: 13.844 on 2 degrees of freedom + AIC: NA + + Number of Fisher Scoring iterations: 11 + + residuals(model, type="pearson") + 1 2 3 4 + 0.7383616 -0.0509458 2.2348337 -1.4552090 + residuals(model, type="working") + 1 2 3 4 + 0.83354150 -0.04103552 1.55676369 -1.00000000 + residuals(model, type="response") + 1 2 3 4 + 0.45460738 -0.02139574 0.60888055 -0.20392801 + */ + val datasetWithWeight = Seq( + Instance(1.0, 1.0, Vectors.dense(0.0, 5.0)), + Instance(0.5, 2.0, Vectors.dense(1.0, 2.0)), + Instance(1.0, 3.0, Vectors.dense(2.0, 1.0)), + Instance(0.0, 4.0, Vectors.dense(3.0, 3.0)) + ).toDF() + + val trainer = new GeneralizedLinearRegression() + .setFamily("tweedie") + .setVariancePower(1.6) + .setLinkPower(0.0) + .setWeightCol("weight") + .setFitIntercept(false) + + val model = trainer.fit(datasetWithWeight) + val coefficientsR = Vectors.dense(Array(-0.408746, -0.12125)) + val interceptR = 0.0 + val devianceResidualsR = Array(0.621047, -0.051515, 1.693473, -3.253946) + val pearsonResidualsR = Array(0.738362, -0.050946, 2.234834, -1.455209) + val workingResidualsR = Array(0.833541, -0.041036, 1.556764, -1.0) + val responseResidualsR = Array(0.454607, -0.021396, 0.608881, -0.203928) + val seCoefR = Array(0.520519, 0.408215) + val tValsR = Array(-0.785267, -0.297024) + val pValsR = Array(0.514549, 0.794457) + val dispersionR = 3.830036 + val nullDevianceR = 20.702 + val residualDevianceR = 13.844 + val residualDegreeOfFreedomNullR = 4 + val residualDegreeOfFreedomR = 2 + + val summary = model.summary + + val devianceResiduals = summary.residuals() + .select(col("devianceResiduals")) + .collect() + .map(_.getDouble(0)) + val pearsonResiduals = summary.residuals("pearson") + .select(col("pearsonResiduals")) + .collect() + .map(_.getDouble(0)) + val workingResiduals = summary.residuals("working") + .select(col("workingResiduals")) + .collect() + .map(_.getDouble(0)) + val responseResiduals = summary.residuals("response") + .select(col("responseResiduals")) + .collect() + .map(_.getDouble(0)) + + assert(model.coefficients ~== coefficientsR absTol 1E-3) + assert(model.intercept ~== interceptR absTol 1E-3) + devianceResiduals.zip(devianceResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + pearsonResiduals.zip(pearsonResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + workingResiduals.zip(workingResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + responseResiduals.zip(responseResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + + summary.coefficientStandardErrors.zip(seCoefR).foreach{ x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + + assert(summary.dispersion ~== dispersionR absTol 1E-3) + assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) + assert(summary.deviance ~== residualDevianceR absTol 1E-3) + assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) + assert(summary.residualDegreeOfFreedomNull === residualDegreeOfFreedomNullR) + assert(summary.solver === "irls") + } + test("glm handle collinear features") { val collinearInstances = Seq( Instance(1.0, 1.0, Vectors.dense(1.0, 2.0)), @@ -1086,7 +1366,7 @@ class GeneralizedLinearRegressionSuite GeneralizedLinearRegressionSuite.allParamSettings, checkModelData) } - test("should support all NumericType labels and not support other types") { + test("should support all NumericType labels and weights, and not support other types") { val glr = new GeneralizedLinearRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[ GeneralizedLinearRegressionModel, GeneralizedLinearRegression]( @@ -1183,7 +1463,8 @@ object GeneralizedLinearRegressionSuite { "maxIter" -> 2, // intentionally small "tol" -> 0.8, "regParam" -> 0.01, - "predictionCol" -> "myPrediction") + "predictionCol" -> "myPrediction", + "variancePower" -> 1.0) def generateGeneralizedLinearRegressionInput( intercept: Double, diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala index c2c79476e8b2b..8cbb2acad243e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala @@ -181,7 +181,7 @@ class IsotonicRegressionSuite checkModelData) } - test("should support all NumericType labels and not support other types") { + test("should support all NumericType labels and weights, and not support other types") { val ir = new IsotonicRegression() MLTestingUtils.checkNumericTypes[IsotonicRegressionModel, IsotonicRegression]( ir, spark, isClassification = false) { (expected, actual) => diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index e05d0c941118e..584a1b272f6c8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -988,7 +988,7 @@ class LinearRegressionSuite checkModelData) } - test("should support all NumericType labels and not support other types") { + test("should support all NumericType labels and weights, and not support other types") { for (solver <- Seq("auto", "l-bfgs", "normal")) { val lr = new LinearRegression().setMaxIter(1).setSolver(solver) MLTestingUtils.checkNumericTypes[LinearRegressionModel, LinearRegression]( diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala index 3bded9c01760a..e1ab7c2d6520b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala @@ -26,9 +26,8 @@ import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.tree._ import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.tree.{DecisionTreeSuite => OldDTSuite, EnsembleTestHelper} -import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, QuantileStrategy, - Strategy => OldStrategy} -import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, GiniCalculator} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, QuantileStrategy, Strategy => OldStrategy} +import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, GiniCalculator, Variance} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.util.collection.OpenHashMap @@ -161,6 +160,21 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } } + test("train with empty arrays") { + val lp = LabeledPoint(1.0, Vectors.dense(Array.empty[Double])) + val data = Array.fill(5)(lp) + val rdd = sc.parallelize(data) + + val strategy = new OldStrategy(OldAlgo.Regression, Gini, maxDepth = 2, + maxBins = 5) + withClue("DecisionTree requires number of features > 0," + + " but was given an empty features vector") { + intercept[IllegalArgumentException] { + RandomForest.run(rdd, strategy, 1, "all", 42L, instr = None) + } + } + } + test("train with constant features") { val lp = LabeledPoint(1.0, Vectors.dense(0.0, 0.0, 0.0)) val data = Array.fill(5)(lp) @@ -170,12 +184,23 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { Gini, maxDepth = 2, numClasses = 2, - maxBins = 100, + maxBins = 5, categoricalFeaturesInfo = Map(0 -> 1, 1 -> 5)) val Array(tree) = RandomForest.run(rdd, strategy, 1, "all", 42L, instr = None) assert(tree.rootNode.impurity === -1.0) assert(tree.depth === 0) assert(tree.rootNode.prediction === lp.label) + + // Test with no categorical features + val strategy2 = new OldStrategy( + OldAlgo.Regression, + Variance, + maxDepth = 2, + maxBins = 5) + val Array(tree2) = RandomForest.run(rdd, strategy2, 1, "all", 42L, instr = None) + assert(tree2.rootNode.impurity === -1.0) + assert(tree2.depth === 0) + assert(tree2.rootNode.prediction === lp.label) } test("Multiclass classification with unordered categorical features: split calculations") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index d219c42818924..f1ed568d5e60a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -47,18 +47,44 @@ object MLTestingUtils extends SparkFunSuite { } else { genRegressionDFWithNumericLabelCol(spark) } - val expected = estimator.fit(dfs(DoubleType)) - val actuals = dfs.keys.filter(_ != DoubleType).map(t => estimator.fit(dfs(t))) + + val finalEstimator = estimator match { + case weighted: Estimator[M] with HasWeightCol => + weighted.set(weighted.weightCol, "weight") + weighted + case _ => estimator + } + + val expected = finalEstimator.fit(dfs(DoubleType)) + + val actuals = dfs.keys.filter(_ != DoubleType).map { t => + finalEstimator.fit(dfs(t)) + } + actuals.foreach(actual => check(expected, actual)) val dfWithStringLabels = spark.createDataFrame(Seq( - ("0", Vectors.dense(0, 2, 3), 0.0) - )).toDF("label", "features", "censor") + ("0", 1, Vectors.dense(0, 2, 3), 0.0) + )).toDF("label", "weight", "features", "censor") val thrown = intercept[IllegalArgumentException] { estimator.fit(dfWithStringLabels) } assert(thrown.getMessage.contains( "Column label must be of type NumericType but was actually of type StringType")) + + estimator match { + case weighted: Estimator[M] with HasWeightCol => + val dfWithStringWeights = spark.createDataFrame(Seq( + (0, "1", Vectors.dense(0, 2, 3), 0.0) + )).toDF("label", "weight", "features", "censor") + weighted.set(weighted.weightCol, "weight") + val thrown = intercept[IllegalArgumentException] { + weighted.fit(dfWithStringWeights) + } + assert(thrown.getMessage.contains( + "Column weight must be of type NumericType but was actually of type StringType")) + case _ => + } } def checkNumericTypesALS( @@ -75,7 +101,7 @@ object MLTestingUtils extends SparkFunSuite { actuals.foreach { case (t, actual) => check2(expected, actual, dfs(t)) } val baseDF = dfs(baseType) - val others = baseDF.columns.toSeq.diff(Seq(column)).map(col(_)) + val others = baseDF.columns.toSeq.diff(Seq(column)).map(col) val cols = Seq(col(column).cast(StringType)) ++ others val strDF = baseDF.select(cols: _*) val thrown = intercept[IllegalArgumentException] { @@ -104,7 +130,8 @@ object MLTestingUtils extends SparkFunSuite { def genClassifDFWithNumericLabelCol( spark: SparkSession, labelColName: String = "label", - featuresColName: String = "features"): Map[NumericType, DataFrame] = { + featuresColName: String = "features", + weightColName: String = "weight"): Map[NumericType, DataFrame] = { val df = spark.createDataFrame(Seq( (0, Vectors.dense(0, 2, 3)), (1, Vectors.dense(0, 3, 1)), @@ -118,12 +145,14 @@ object MLTestingUtils extends SparkFunSuite { types.map { t => val castDF = df.select(col(labelColName).cast(t), col(featuresColName)) t -> TreeTests.setMetadata(castDF, 2, labelColName, featuresColName) + .withColumn(weightColName, round(rand(seed = 42)).cast(t)) }.toMap } def genRegressionDFWithNumericLabelCol( spark: SparkSession, labelColName: String = "label", + weightColName: String = "weight", featuresColName: String = "features", censorColName: String = "censor"): Map[NumericType, DataFrame] = { val df = spark.createDataFrame(Seq( @@ -137,10 +166,11 @@ object MLTestingUtils extends SparkFunSuite { val types = Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) types.map { t => - val castDF = df.select(col(labelColName).cast(t), col(featuresColName)) - t -> TreeTests.setMetadata(castDF, 0, labelColName, featuresColName) - .withColumn(censorColName, lit(0.0)) - }.toMap + val castDF = df.select(col(labelColName).cast(t), col(featuresColName)) + t -> TreeTests.setMetadata(castDF, 0, labelColName, featuresColName) + .withColumn(censorColName, lit(0.0)) + .withColumn(weightColName, round(rand(seed = 42)).cast(t)) + }.toMap } def genRatingsDFWithNumericCols( @@ -154,7 +184,7 @@ object MLTestingUtils extends SparkFunSuite { (4, 50, 5.0) )).toDF("user", "item", "rating") - val others = df.columns.toSeq.diff(Seq(column)).map(col(_)) + val others = df.columns.toSeq.diff(Seq(column)).map(col) val types: Seq[NumericType] = Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) types.map { t => diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala index 67e680be73303..11189d8bd4776 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala @@ -25,6 +25,20 @@ import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext { + + test("gmm fails on high dimensional data") { + val rdd = sc.parallelize(Seq( + Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(0, 4), Array(3.0, 8.0)), + Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(1, 5), Array(4.0, 9.0)))) + val gm = new GaussianMixture() + withClue(s"GMM should restrict the maximum number of features to be < " + + s"${GaussianMixture.MAX_NUM_FEATURES}") { + intercept[IllegalArgumentException] { + gm.run(rdd) + } + } + } + test("single cluster") { val data = sc.parallelize(Array( Vectors.dense(6.0, 9.0), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index 61266f3c78dbc..f6a996940291c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -267,6 +267,15 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(sparseBM.subtract(sparseBM).toBreeze() === sparseBM.subtract(denseBM).toBreeze()) } + def testMultiply(A: BlockMatrix, B: BlockMatrix, expectedResult: Matrix, + numMidDimSplits: Int): Unit = { + val C = A.multiply(B, numMidDimSplits) + val localC = C.toLocalMatrix() + assert(C.numRows() === A.numRows()) + assert(C.numCols() === B.numCols()) + assert(localC ~== expectedResult absTol 1e-8) + } + test("multiply") { // identity matrix val blocks: Seq[((Int, Int), Matrix)] = Seq( @@ -302,12 +311,13 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { // Try it with increased number of partitions val largeA = new BlockMatrix(sc.parallelize(largerAblocks, 10), 6, 4) val largeB = new BlockMatrix(sc.parallelize(largerBblocks, 8), 4, 4) - val largeC = largeA.multiply(largeB) - val localC = largeC.toLocalMatrix() + val result = largeA.toLocalMatrix().multiply(largeB.toLocalMatrix().asInstanceOf[DenseMatrix]) - assert(largeC.numRows() === largeA.numRows()) - assert(largeC.numCols() === largeB.numCols()) - assert(localC ~== result absTol 1e-8) + + testMultiply(largeA, largeB, result, 1) + testMultiply(largeA, largeB, result, 2) + testMultiply(largeA, largeB, result, 3) + testMultiply(largeA, largeB, result, 4) } test("simulate multiply") { @@ -318,7 +328,7 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val B = new BlockMatrix(rdd, colPerPart, rowPerPart) val resultPartitioner = GridPartitioner(gridBasedMat.numRowBlocks, B.numColBlocks, math.max(numPartitions, 2)) - val (destinationsA, destinationsB) = gridBasedMat.simulateMultiply(B, resultPartitioner) + val (destinationsA, destinationsB) = gridBasedMat.simulateMultiply(B, resultPartitioner, 1) assert(destinationsA((0, 0)) === Set(0)) assert(destinationsA((0, 1)) === Set(2)) assert(destinationsA((1, 0)) === Set(0)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala index 94da626d92ebb..02ea74b87f684 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.regression import org.scalatest.Matchers -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils @@ -163,17 +163,16 @@ class IsotonicRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w } test("weighted isotonic regression with negative weights") { - val model = runIsotonicRegression(Seq(1, 2, 3, 2, 1), Seq(-1, 1, -3, 1, -5), true) - - assert(model.boundaries === Array(0.0, 1.0, 4.0)) - assert(model.predictions === Array(1.0, 10.0/6, 10.0/6)) + val ex = intercept[SparkException] { + runIsotonicRegression(Seq(1, 2, 3, 2, 1), Seq(-1, 1, -3, 1, -5), true) + } + assert(ex.getCause.isInstanceOf[IllegalArgumentException]) } test("weighted isotonic regression with zero weights") { - val model = runIsotonicRegression(Seq[Double](1, 2, 3, 2, 1), Seq[Double](0, 0, 0, 1, 0), true) - - assert(model.boundaries === Array(0.0, 1.0, 4.0)) - assert(model.predictions === Array(1, 2, 2)) + val model = runIsotonicRegression(Seq(1, 2, 3, 2, 1, 0), Seq(0, 0, 0, 1, 1, 0), true) + assert(model.boundaries === Array(3, 4)) + assert(model.predictions === Array(1.5, 1.5)) } test("SPARK-16426 isotonic regression with duplicate features that produce NaNs") { diff --git a/pom.xml b/pom.xml index ff37a88db02f7..3679b4e263619 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,7 @@ 1.2.1 10.12.1.1 - 1.8.1 + 1.8.2 1.6.0 9.2.16.v20160414 3.1.0 @@ -2571,7 +2571,7 @@ hadoop-2.6 - 2.6.4 + 2.6.5 0.9.3 3.4.6 2.6.0 diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 2314d7f45cb21..7e6e143523387 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -43,7 +43,18 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.recoverPartitions"), // [SPARK-18537] Add a REST api to spark streaming - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.streaming.scheduler.StreamingListener.onStreamingStarted") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.streaming.scheduler.StreamingListener.onStreamingStarted"), + + // [SPARK-19148][SQL] do not expose the external table concept in Catalog + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createTable"), + + // [SPARK-14272][ML] Add logLikelihood in GaussianMixtureSummary + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.clustering.GaussianMixtureSummary.this"), + + // [SPARK-19069] [CORE] Expose task 'status' and 'duration' in spark history server REST API. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$10"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$11") ) // Exclude rules for 2.1.x diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index da2b2f3757967..959fb8b357f99 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -43,6 +43,7 @@ from __future__ import print_function import operator +import opcode import os import io import pickle @@ -53,6 +54,8 @@ import itertools import dis import traceback +import weakref + if sys.version < '3': from pickle import Pickler @@ -68,10 +71,10 @@ PY3 = True #relevant opcodes -STORE_GLOBAL = dis.opname.index('STORE_GLOBAL') -DELETE_GLOBAL = dis.opname.index('DELETE_GLOBAL') -LOAD_GLOBAL = dis.opname.index('LOAD_GLOBAL') -GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] +STORE_GLOBAL = opcode.opmap['STORE_GLOBAL'] +DELETE_GLOBAL = opcode.opmap['DELETE_GLOBAL'] +LOAD_GLOBAL = opcode.opmap['LOAD_GLOBAL'] +GLOBAL_OPS = (STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL) HAVE_ARGUMENT = dis.HAVE_ARGUMENT EXTENDED_ARG = dis.EXTENDED_ARG @@ -90,6 +93,43 @@ def _builtin_type(name): return getattr(types, name) +if sys.version_info < (3, 4): + def _walk_global_ops(code): + """ + Yield (opcode, argument number) tuples for all + global-referencing instructions in *code*. + """ + code = getattr(code, 'co_code', b'') + if not PY3: + code = map(ord, code) + + n = len(code) + i = 0 + extended_arg = 0 + while i < n: + op = code[i] + i += 1 + if op >= HAVE_ARGUMENT: + oparg = code[i] + code[i + 1] * 256 + extended_arg + extended_arg = 0 + i += 2 + if op == EXTENDED_ARG: + extended_arg = oparg * 65536 + if op in GLOBAL_OPS: + yield op, oparg + +else: + def _walk_global_ops(code): + """ + Yield (opcode, argument number) tuples for all + global-referencing instructions in *code*. + """ + for instr in dis.get_instructions(code): + op = instr.opcode + if op in GLOBAL_OPS: + yield op, instr.arg + + class CloudPickler(Pickler): dispatch = Pickler.dispatch.copy() @@ -260,38 +300,34 @@ def save_function_tuple(self, func): write(pickle.TUPLE) write(pickle.REDUCE) # applies _fill_function on the tuple - @staticmethod - def extract_code_globals(co): + _extract_code_globals_cache = ( + weakref.WeakKeyDictionary() + if sys.version_info >= (2, 7) and not hasattr(sys, "pypy_version_info") + else {}) + + @classmethod + def extract_code_globals(cls, co): """ Find all globals names read or written to by codeblock co """ - code = co.co_code - if not PY3: - code = [ord(c) for c in code] - names = co.co_names - out_names = set() - - n = len(code) - i = 0 - extended_arg = 0 - while i < n: - op = code[i] + out_names = cls._extract_code_globals_cache.get(co) + if out_names is None: + try: + names = co.co_names + except AttributeError: + # PyPy "builtin-code" object + out_names = set() + else: + out_names = set(names[oparg] + for op, oparg in _walk_global_ops(co)) - i += 1 - if op >= HAVE_ARGUMENT: - oparg = code[i] + code[i+1] * 256 + extended_arg - extended_arg = 0 - i += 2 - if op == EXTENDED_ARG: - extended_arg = oparg*65536 - if op in GLOBAL_OPS: - out_names.add(names[oparg]) + # see if nested function have any global refs + if co.co_consts: + for const in co.co_consts: + if type(const) is types.CodeType: + out_names |= cls.extract_code_globals(const) - # see if nested function have any global refs - if co.co_consts: - for const in co.co_consts: - if type(const) is types.CodeType: - out_names |= CloudPickler.extract_code_globals(const) + cls._extract_code_globals_cache[co] = out_names return out_names diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 5c4e79cb0499e..ac4b2b035f5c1 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -132,6 +132,9 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self._conf = conf else: self._conf = SparkConf(_jvm=SparkContext._jvm) + if conf is not None: + for k, v in conf.getAll(): + self._conf.set(k, v) self._batchSize = batchSize # -1 represents an unlimited batch size self._unbatched_serializer = serializer diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 5fe4bab186bd2..f10556ca92290 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -31,7 +31,8 @@ from pyspark.sql.types import ArrayType, DoubleType from pyspark.storagelevel import StorageLevel -__all__ = ['LogisticRegression', 'LogisticRegressionModel', +__all__ = ['LinearSVC', 'LinearSVCModel', + 'LogisticRegression', 'LogisticRegressionModel', 'LogisticRegressionSummary', 'LogisticRegressionTrainingSummary', 'BinaryLogisticRegressionSummary', 'BinaryLogisticRegressionTrainingSummary', 'DecisionTreeClassifier', 'DecisionTreeClassificationModel', @@ -59,6 +60,134 @@ def numClasses(self): return self._call_java("numClasses") +@inherit_doc +class LinearSVC(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, + HasRegParam, HasTol, HasRawPredictionCol, HasFitIntercept, HasStandardization, + HasThreshold, HasWeightCol, HasAggregationDepth, JavaMLWritable, JavaMLReadable): + """ + `Linear SVM Classifier `_ + This binary classifier optimizes the Hinge Loss using the OWLQN optimizer. + + >>> from pyspark.sql import Row + >>> from pyspark.ml.linalg import Vectors + >>> df = sc.parallelize([ + ... Row(label=1.0, features=Vectors.dense(1.0, 1.0, 1.0)), + ... Row(label=0.0, features=Vectors.dense(1.0, 2.0, 3.0))]).toDF() + >>> svm = LinearSVC(maxIter=5, regParam=0.01) + >>> model = svm.fit(df) + >>> model.coefficients + DenseVector([0.0, -0.2792, -0.1833]) + >>> model.intercept + 1.0206118982229047 + >>> model.numClasses + 2 + >>> model.numFeatures + 3 + >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0, -1.0, -1.0))]).toDF() + >>> result = model.transform(test0).head() + >>> result.prediction + 1.0 + >>> result.rawPrediction + DenseVector([-1.4831, 1.4831]) + >>> svm.setParams("vector") + Traceback (most recent call last): + ... + TypeError: Method setParams forces keyword arguments. + >>> svm_path = temp_path + "/svm" + >>> svm.save(svm_path) + >>> svm2 = LinearSVC.load(svm_path) + >>> svm2.getMaxIter() + 5 + >>> model_path = temp_path + "/svm_model" + >>> model.save(model_path) + >>> model2 = LinearSVCModel.load(model_path) + >>> model.coefficients[0] == model2.coefficients[0] + True + >>> model.intercept == model2.intercept + True + + .. versionadded:: 2.2.0 + """ + + @keyword_only + def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", + maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", + fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, + aggregationDepth=2): + """ + __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ + maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", \ + fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, \ + aggregationDepth=2): + """ + super(LinearSVC, self).__init__() + self._java_obj = self._new_java_obj( + "org.apache.spark.ml.classification.LinearSVC", self.uid) + self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, + standardization=True, threshold=0.0, aggregationDepth=2) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + @since("2.2.0") + def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", + maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", + fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, + aggregationDepth=2): + """ + setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ + maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", \ + fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, \ + aggregationDepth=2): + Sets params for Linear SVM Classifier. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def _create_model(self, java_model): + return LinearSVCModel(java_model) + + +class LinearSVCModel(JavaModel, JavaClassificationModel, JavaMLWritable, JavaMLReadable): + """ + Model fitted by LinearSVC. + + .. versionadded:: 2.2.0 + """ + + @property + @since("2.2.0") + def coefficients(self): + """ + Model coefficients of Linear SVM Classifier. + """ + return self._call_java("coefficients") + + @property + @since("2.2.0") + def intercept(self): + """ + Model intercept of Linear SVM Classifier. + """ + return self._call_java("intercept") + + @property + @since("2.2.0") + def numClasses(self): + """ + Number of classes. + """ + return self._call_java("numClasses") + + @property + @since("2.2.0") + def numFeatures(self): + """ + Number of features. + """ + return self._call_java("numFeatures") + + @inherit_doc class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, HasRegParam, HasTol, HasProbabilityCol, HasRawPredictionCol, diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 25f97f5696a14..c6c1a0033190e 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -175,6 +175,8 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte 3 >>> summary.clusterSizes [2, 2, 2] + >>> summary.logLikelihood + 8.14636... >>> weights = model.weights >>> len(weights) 3 @@ -281,6 +283,14 @@ def probability(self): """ return self._call_java("probability") + @property + @since("2.2.0") + def logLikelihood(self): + """ + Total log-likelihood for this model on the given data. + """ + return self._call_java("logLikelihood") + class KMeansSummary(ClusteringSummary): """ diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 929591236d688..51d49b524c326 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -143,6 +143,8 @@ def get$Name(self): "The class with largest value p/t is predicted, where p is the original " + "probability of that class and t is the class's threshold.", None, "TypeConverters.toListFloat"), + ("threshold", "threshold in binary classification prediction, in range [0, 1]", + "0.5", "TypeConverters.toFloat"), ("weightCol", "weight column name. If this is not set or empty, we treat " + "all instance weights as 1.0.", None, "TypeConverters.toString"), ("solver", "the solver algorithm for optimization. If this is not set or empty, " + diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index cc596936d82f6..163a0e2b3a968 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -490,6 +490,30 @@ def getThresholds(self): return self.getOrDefault(self.thresholds) +class HasThreshold(Params): + """ + Mixin for param threshold: threshold in binary classification prediction, in range [0, 1] + """ + + threshold = Param(Params._dummy(), "threshold", "threshold in binary classification prediction, in range [0, 1]", typeConverter=TypeConverters.toFloat) + + def __init__(self): + super(HasThreshold, self).__init__() + self._setDefault(threshold=0.5) + + def setThreshold(self, value): + """ + Sets the value of :py:attr:`threshold`. + """ + return self._set(threshold=value) + + def getThreshold(self): + """ + Gets the value of threshold or its default value. + """ + return self.getOrDefault(self.threshold) + + class HasWeightCol(Params): """ Mixin for param weightCol: weight column name. If this is not set or empty, we treat all instance weights as 1.0. diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index c4f2f08cb4445..ea5e00e9eeef5 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -382,18 +382,38 @@ def _hijack_namedtuple(): return global _old_namedtuple # or it will put in closure + global _old_namedtuple_kwdefaults # or it will put in closure too def _copy_func(f): return types.FunctionType(f.__code__, f.__globals__, f.__name__, f.__defaults__, f.__closure__) + def _kwdefaults(f): + # __kwdefaults__ contains the default values of keyword-only arguments which are + # introduced from Python 3. The possible cases for __kwdefaults__ in namedtuple + # are as below: + # + # - Does not exist in Python 2. + # - Returns None in <= Python 3.5.x. + # - Returns a dictionary containing the default values to the keys from Python 3.6.x + # (See https://bugs.python.org/issue25628). + kargs = getattr(f, "__kwdefaults__", None) + if kargs is None: + return {} + else: + return kargs + _old_namedtuple = _copy_func(collections.namedtuple) + _old_namedtuple_kwdefaults = _kwdefaults(collections.namedtuple) def namedtuple(*args, **kwargs): + for k, v in _old_namedtuple_kwdefaults.items(): + kwargs[k] = kwargs.get(k, v) cls = _old_namedtuple(*args, **kwargs) return _hack_namedtuple(cls) # replace namedtuple with new one + collections.namedtuple.__globals__["_old_namedtuple_kwdefaults"] = _old_namedtuple_kwdefaults collections.namedtuple.__globals__["_old_namedtuple"] = _old_namedtuple collections.namedtuple.__globals__["_hack_namedtuple"] = _hack_namedtuple collections.namedtuple.__code__ = namedtuple.__code__ diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 30c7a3fe4fe60..253a750629170 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -15,6 +15,7 @@ # limitations under the License. # +import warnings from collections import namedtuple from pyspark import since @@ -138,7 +139,27 @@ def listColumns(self, tableName, dbName=None): @since(2.0) def createExternalTable(self, tableName, path=None, source=None, schema=None, **options): - """Creates an external table based on the dataset in a data source. + """Creates a table based on the dataset in a data source. + + It returns the DataFrame associated with the external table. + + The data source is specified by the ``source`` and a set of ``options``. + If ``source`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. + + Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and + created external table. + + :return: :class:`DataFrame` + """ + warnings.warn( + "createExternalTable is deprecated since Spark 2.2, please use createTable instead.", + DeprecationWarning) + return self.createTable(tableName, path, source, schema, **options) + + @since(2.2) + def createTable(self, tableName, path=None, source=None, schema=None, **options): + """Creates a table based on the dataset in a data source. It returns the DataFrame associated with the external table. @@ -157,12 +178,12 @@ def createExternalTable(self, tableName, path=None, source=None, schema=None, ** source = self._sparkSession.conf.get( "spark.sql.sources.default", "org.apache.spark.sql.parquet") if schema is None: - df = self._jcatalog.createExternalTable(tableName, source, options) + df = self._jcatalog.createTable(tableName, source, options) else: if not isinstance(schema, StructType): raise TypeError("schema should be StructType") scala_datatype = self._jsparkSession.parseDataType(schema.json()) - df = self._jcatalog.createExternalTable(tableName, source, scala_datatype, options) + df = self._jcatalog.createTable(tableName, source, scala_datatype, options) return DataFrame(df, self._sparkSession._wrapped) @since(2.0) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 8d5adc8ffd6d4..ec059d6258439 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -27,7 +27,7 @@ from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.types import * -__all__ = ["DataFrame", "Column", "DataFrameNaFunctions", "DataFrameStatFunctions"] +__all__ = ["Column"] def _create_column_from_literal(literal): diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index b0c51b1e9992e..d31f3fb8f6046 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -399,7 +399,8 @@ def jdbc(self, url, table, column=None, lowerBound=None, upperBound=None, numPar accessible via JDBC URL ``url`` and connection ``properties``. Partitions of the table will be retrieved in parallel if either ``column`` or - ``predicates`` is specified. + ``predicates`` is specified. ``lowerBound`, ``upperBound`` and ``numPartitions`` + is needed when ``column`` is specified. If both ``column`` and ``predicates`` are specified, ``column`` will be used. @@ -429,8 +430,10 @@ def jdbc(self, url, table, column=None, lowerBound=None, upperBound=None, numPar for k in properties: jprop.setProperty(k, properties[k]) if column is not None: - if numPartitions is None: - numPartitions = self._spark._sc.defaultParallelism + assert lowerBound is not None, "lowerBound can not be None when ``column`` is specified" + assert upperBound is not None, "upperBound can not be None when ``column`` is specified" + assert numPartitions is not None, \ + "numPartitions can not be None when ``column`` is specified" return self._df(self._jreader.jdbc(url, table, column, int(lowerBound), int(upperBound), int(numPartitions), jprop)) if predicates is not None: diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index a8250281dab35..a88e5a1cfb3c9 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -342,6 +342,15 @@ def test_udf_in_filter_on_top_of_outer_join(self): df = df.withColumn('b', udf(lambda x: 'x')(df.a)) self.assertEqual(df.filter('b = "x"').collect(), [Row(a=1, b='x')]) + def test_udf_in_filter_on_top_of_join(self): + # regression test for SPARK-18589 + from pyspark.sql.functions import udf + left = self.spark.createDataFrame([Row(a=1)]) + right = self.spark.createDataFrame([Row(b=1)]) + f = udf(lambda a, b: a == b, BooleanType()) + df = left.crossJoin(right).filter(f("a", "b")) + self.assertEqual(df.collect(), [Row(a=1, b=1)]) + def test_udf_without_arguments(self): self.spark.catalog.registerFunction("foo", lambda: "bar") [row] = self.spark.sql("SELECT foo()").collect() @@ -435,6 +444,30 @@ def test_udf_with_input_file_name(self): row = self.spark.read.json(filePath).select(sourceFile(input_file_name())).first() self.assertTrue(row[0].find("people1.json") != -1) + def test_udf_with_input_file_name_for_hadooprdd(self): + from pyspark.sql.functions import udf, input_file_name + from pyspark.sql.types import StringType + + def filename(path): + return path + + sameText = udf(filename, StringType()) + + rdd = self.sc.textFile('python/test_support/sql/people.json') + df = self.spark.read.json(rdd).select(input_file_name().alias('file')) + row = df.select(sameText(df['file'])).first() + self.assertTrue(row[0].find("people.json") != -1) + + rdd2 = self.sc.newAPIHadoopFile( + 'python/test_support/sql/people.json', + 'org.apache.hadoop.mapreduce.lib.input.TextInputFormat', + 'org.apache.hadoop.io.LongWritable', + 'org.apache.hadoop.io.Text') + + df2 = self.spark.read.json(rdd2).select(input_file_name().alias('file')) + row2 = df2.select(sameText(df2['file'])).first() + self.assertTrue(row2[0].find("people.json") != -1) + def test_basic_functions(self): rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) df = self.spark.read.json(rdd) @@ -1684,8 +1717,8 @@ def test_list_tables(self): self.assertEquals(spark.catalog.listTables(), []) self.assertEquals(spark.catalog.listTables("some_db"), []) spark.createDataFrame([(1, 1)]).createOrReplaceTempView("temp_tab") - spark.sql("CREATE TABLE tab1 (name STRING, age INT)") - spark.sql("CREATE TABLE some_db.tab2 (name STRING, age INT)") + spark.sql("CREATE TABLE tab1 (name STRING, age INT) USING parquet") + spark.sql("CREATE TABLE some_db.tab2 (name STRING, age INT) USING parquet") tables = sorted(spark.catalog.listTables(), key=lambda t: t.name) tablesDefault = sorted(spark.catalog.listTables("default"), key=lambda t: t.name) tablesSomeDb = sorted(spark.catalog.listTables("some_db"), key=lambda t: t.name) @@ -1763,8 +1796,8 @@ def test_list_columns(self): spark = self.spark spark.catalog._reset() spark.sql("CREATE DATABASE some_db") - spark.sql("CREATE TABLE tab1 (name STRING, age INT)") - spark.sql("CREATE TABLE some_db.tab2 (nickname STRING, tolerance FLOAT)") + spark.sql("CREATE TABLE tab1 (name STRING, age INT) USING parquet") + spark.sql("CREATE TABLE some_db.tab2 (nickname STRING, tolerance FLOAT) USING parquet") columns = sorted(spark.catalog.listColumns("tab1"), key=lambda c: c.name) columnsDefault = sorted(spark.catalog.listColumns("tab1", "default"), key=lambda c: c.name) self.assertEquals(columns, columnsDefault) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 5ac007cd598b9..2e8ed698278d0 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -1420,7 +1420,7 @@ def test_kinesis_stream(self): import random kinesisAppName = ("KinesisStreamTests-%d" % abs(random.randint(0, 10000000))) - kinesisTestUtils = self.ssc._jvm.org.apache.spark.streaming.kinesis.KinesisTestUtils() + kinesisTestUtils = self.ssc._jvm.org.apache.spark.streaming.kinesis.KinesisTestUtils(2) try: kinesisTestUtils.createStream() aWSCredentials = kinesisTestUtils.getAWSCredentials() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index c383d9ab6767d..e908b1e739bb3 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -2035,6 +2035,26 @@ def test_single_script_on_cluster(self): self.assertEqual(0, proc.returncode) self.assertIn("[2, 4, 6]", out.decode('utf-8')) + def test_user_configuration(self): + """Make sure user configuration is respected (SPARK-19307)""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkConf, SparkContext + | + |conf = SparkConf().set("spark.test_config", "1") + |sc = SparkContext(conf = conf) + |try: + | if sc._conf.get("spark.test_config") != "1": + | raise Exception("Cannot find spark.test_config in SparkContext's conf.") + |finally: + | sc.stop() + """) + proc = subprocess.Popen( + [self.sparkSubmit, "--master", "local", script], + stdout=subprocess.PIPE, + stderr=subprocess.STDOUT) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode, msg="Process failed with error:\n {0}".format(out)) + class ContextTests(unittest.TestCase): diff --git a/python/setup.py b/python/setup.py index bc2eb4ce9dbd0..47eab98e0f7b3 100644 --- a/python/setup.py +++ b/python/setup.py @@ -162,7 +162,12 @@ def _supports_symlinks(): url='https://github.com/apache/spark/tree/master/python', packages=['pyspark', 'pyspark.mllib', + 'pyspark.mllib.linalg', + 'pyspark.mllib.stat', 'pyspark.ml', + 'pyspark.ml.linalg', + 'pyspark.ml.param', + 'pyspark.ml.stat', 'pyspark.sql', 'pyspark.streaming', 'pyspark.bin', diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala index 24fbbc12c08da..be9b79021d2a8 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala @@ -17,8 +17,8 @@ package org.apache.spark.repl -import scala.tools.nsc.{Settings, CompilerCommand} -import scala.Predef._ +import scala.tools.nsc.{CompilerCommand, Settings} + import org.apache.spark.annotation.DeveloperApi /** diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index b2a61260c2bb6..5f0d92bccd809 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -10,8 +10,6 @@ package org.apache.spark.repl import scala.tools.nsc._ import scala.tools.nsc.interpreter._ -import scala.reflect.internal.util.Position -import scala.util.control.Exception.ignoring import scala.tools.nsc.util.stackTraceString import org.apache.spark.SPARK_VERSION diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 3258b09c06278..f555072c3842a 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -136,8 +136,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( new MesosExternalShuffleClient( SparkTransportConf.fromSparkConf(conf, "shuffle"), securityManager, - securityManager.isAuthenticationEnabled(), - securityManager.isSaslEncryptionEnabled()) + securityManager.isAuthenticationEnabled()) } var nextMesosTaskId = 0 diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 5cdec87667a5d..cc76a7c8f13f5 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -19,8 +19,6 @@ package org.apache.spark.deploy.yarn import scala.collection.mutable.ArrayBuffer -import org.apache.spark.util.{IntParam, MemoryParam} - class ApplicationMasterArguments(val args: Array[String]) { var userJar: String = null var userClass: String = null diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala index fb2d61f621c8a..f2b6324db619a 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala @@ -129,9 +129,9 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy( val largestRatio = updatedHostToContainerCount.values.max // Round the ratio of preferred locality to the number of locality required container // number, which is used for locality preferred host calculating. - var preferredLocalityRatio = updatedHostToContainerCount.mapValues { ratio => + var preferredLocalityRatio = updatedHostToContainerCount.map { case(k, ratio) => val adjustedRatio = ratio.toDouble * requiredLocalityAwareContainerNum / largestRatio - adjustedRatio.ceil.toInt + (k, adjustedRatio.ceil.toInt) } for (i <- 0 until requiredLocalityAwareContainerNum) { @@ -145,7 +145,7 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy( // Minus 1 each time when the host is used. When the current ratio is 0, // which means all the required ratio is satisfied, this host will not be allocated again. - preferredLocalityRatio = preferredLocalityRatio.mapValues(_ - 1) + preferredLocalityRatio = preferredLocalityRatio.map { case (k, v) => (k, v - 1) } } } @@ -218,7 +218,8 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy( val possibleTotalContainerNum = pendingHostToContainerCount.values.sum val localityMatchedPendingNum = localityMatchedPendingAllocations.size.toDouble - pendingHostToContainerCount.mapValues(_ * localityMatchedPendingNum / possibleTotalContainerNum) - .toMap + pendingHostToContainerCount.map { case (k, v) => + (k, v * localityMatchedPendingNum / possibleTotalContainerNum) + }.toMap } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index 666cb456a9be0..f19a5b22a757d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -248,6 +248,11 @@ package object config { .toSequence .createWithDefault(Nil) + private[spark] val FILESYSTEMS_TO_ACCESS = ConfigBuilder("spark.yarn.access.hadoopFileSystems") + .doc("Extra Hadoop filesystem URLs for which to request delegation tokens. The filesystem " + + "that hosts fs.defaultFS does not need to be listed here.") + .fallbackConf(NAMENODES_TO_ACCESS) + /* Rolled log aggregation configuration. */ private[spark] val ROLLED_LOG_INCLUDE_PATTERN = diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala index b4fb4a790adc5..f65c886db944e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala @@ -47,7 +47,7 @@ private[security] class HadoopFSCredentialProvider // NameNode to access, used to get tokens from different FileSystems val tmpCreds = new Credentials() val tokenRenewer = getTokenRenewer(hadoopConf) - nnsToAccess(hadoopConf, sparkConf).foreach { dst => + hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst => val dstFs = dst.getFileSystem(hadoopConf) logInfo("getting token for: " + dst) dstFs.addDelegationTokens(tokenRenewer, tmpCreds) @@ -80,7 +80,7 @@ private[security] class HadoopFSCredentialProvider // user as renewer. sparkConf.get(PRINCIPAL).flatMap { renewer => val creds = new Credentials() - nnsToAccess(hadoopConf, sparkConf).foreach { dst => + hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst => val dstFs = dst.getFileSystem(hadoopConf) dstFs.addDelegationTokens(renewer, creds) } @@ -112,8 +112,8 @@ private[security] class HadoopFSCredentialProvider delegTokenRenewer } - private def nnsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { - sparkConf.get(NAMENODES_TO_ACCESS).map(new Path(_)).toSet + + private def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { + sparkConf.get(FILESYSTEMS_TO_ACCESS).map(new Path(_)).toSet + sparkConf.get(STAGING_DIR).map(new Path(_)) .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala new file mode 100644 index 0000000000000..fb80ff9f31322 --- /dev/null +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import scala.collection.mutable.{HashMap, HashSet, Set} + +import org.apache.hadoop.fs.CommonConfigurationKeysPublic +import org.apache.hadoop.net.DNSToSwitchMapping +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.mockito.Mockito._ + +import org.apache.spark.{SparkConf, SparkFunSuite} + +class LocalityPlacementStrategySuite extends SparkFunSuite { + + test("handle large number of containers and tasks (SPARK-18750)") { + // Run the test in a thread with a small stack size, since the original issue + // surfaced as a StackOverflowError. + var error: Throwable = null + + val runnable = new Runnable() { + override def run(): Unit = try { + runTest() + } catch { + case e: Throwable => error = e + } + } + + val thread = new Thread(new ThreadGroup("test"), runnable, "test-thread", 32 * 1024) + thread.start() + thread.join() + + assert(error === null) + } + + private def runTest(): Unit = { + val yarnConf = new YarnConfiguration() + yarnConf.setClass( + CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + classOf[MockResolver], classOf[DNSToSwitchMapping]) + + // The numbers below have been chosen to balance being large enough to replicate the + // original issue while not taking too long to run when the issue is fixed. The main + // goal is to create enough requests for localized containers (so there should be many + // tasks on several hosts that have no allocated containers). + + val resource = Resource.newInstance(8 * 1024, 4) + val strategy = new LocalityPreferredContainerPlacementStrategy(new SparkConf(), + yarnConf, resource) + + val totalTasks = 32 * 1024 + val totalContainers = totalTasks / 16 + val totalHosts = totalContainers / 16 + + val mockId = mock(classOf[ContainerId]) + val hosts = (1 to totalHosts).map { i => (s"host_$i", totalTasks % i) }.toMap + val containers = (1 to totalContainers).map { i => mockId } + val count = containers.size / hosts.size / 2 + + val hostToContainerMap = new HashMap[String, Set[ContainerId]]() + hosts.keys.take(hosts.size / 2).zipWithIndex.foreach { case (host, i) => + val hostContainers = new HashSet[ContainerId]() + containers.drop(count * i).take(i).foreach { c => hostContainers += c } + hostToContainerMap(host) = hostContainers + } + + strategy.localityOfRequestedContainers(containers.size * 2, totalTasks, hosts, + hostToContainerMap, Nil) + } + +} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala index 0eb25127238c6..f50ee193c258f 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala @@ -18,10 +18,9 @@ package org.apache.spark.deploy.yarn.security import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path import org.scalatest.{Matchers, PrivateMethodTester} -import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkFunSuite} class HadoopFSCredentialProviderSuite extends SparkFunSuite diff --git a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala similarity index 100% rename from yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala rename to resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala index b805cfe88be63..0b6fa56469704 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst +import java.util.TimeZone + import org.apache.spark.sql.catalyst.analysis._ /** @@ -36,6 +38,8 @@ trait CatalystConf { def warehousePath: String + def sessionLocalTimeZone: String + /** If true, cartesian products between relations will be allowed for all * join types(inner, (left|right|full) outer). * If false, cartesian products will require explicit CROSS JOIN syntax. @@ -68,5 +72,6 @@ case class SimpleCatalystConf( runSQLonFile: Boolean = true, crossJoinEnabled: Boolean = false, cboEnabled: Boolean = false, - warehousePath: String = "/user/hive/warehouse") + warehousePath: String = "/user/hive/warehouse", + sessionLocalTimeZone: String = TimeZone.getDefault().getID) extends CatalystConf diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index bd9037ec4333f..9e6dbf3344064 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -106,6 +106,13 @@ class Analyzer( */ val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil + /** + * Override to provide rules to do post-hoc resolution. Note that these rules will be executed + * in an individual batch. This batch is to run right after the normal resolution batch and + * execute its rules in one pass. + */ + val postHocResolutionRules: Seq[Rule[LogicalPlan]] = Nil + lazy val batches: Seq[Batch] = Seq( Batch("Substitution", fixedPoint, CTESubstitution, @@ -139,6 +146,7 @@ class Analyzer( ResolveInlineTables :: TypeCoercion.typeCoercionRules ++ extendedResolutionRules : _*), + Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*), Batch("View", Once, AliasViewChild(conf)), Batch("Nondeterministic", Once, @@ -147,6 +155,8 @@ class Analyzer( HandleNullInputsForUDF), Batch("FixNullability", Once, FixNullability), + Batch("ResolveTimeZone", Once, + ResolveTimeZone), Batch("Cleanup", fixedPoint, CleanupAliases) ) @@ -215,7 +225,7 @@ class Analyzer( case ne: NamedExpression => ne case e if !e.resolved => u case g: Generator => MultiAlias(g, Nil) - case c @ Cast(ne: NamedExpression, _) => Alias(c, ne.name)() + case c @ Cast(ne: NamedExpression, _, _) => Alias(c, ne.name)() case e: ExtractValue => Alias(e, toPrettySQL(e))() case e if optGenAliasFunc.isDefined => Alias(child, optGenAliasFunc.get.apply(e))() @@ -1619,11 +1629,18 @@ class Analyzer( case _ => expr } - /** Extracts a [[Generator]] expression and any names assigned by aliases to their output. */ private object AliasedGenerator { - def unapply(e: Expression): Option[(Generator, Seq[String])] = e match { - case Alias(g: Generator, name) if g.resolved => Some((g, name :: Nil)) - case MultiAlias(g: Generator, names) if g.resolved => Some(g, names) + /** + * Extracts a [[Generator]] expression, any names assigned by aliases to the outputs + * and the outer flag. The outer flag is used when joining the generator output. + * @param e the [[Expression]] + * @return (the [[Generator]], seq of output names, outer flag) + */ + def unapply(e: Expression): Option[(Generator, Seq[String], Boolean)] = e match { + case Alias(GeneratorOuter(g: Generator), name) if g.resolved => Some((g, name :: Nil, true)) + case MultiAlias(GeneratorOuter(g: Generator), names) if g.resolved => Some(g, names, true) + case Alias(g: Generator, name) if g.resolved => Some((g, name :: Nil, false)) + case MultiAlias(g: Generator, names) if g.resolved => Some(g, names, false) case _ => None } } @@ -1644,7 +1661,8 @@ class Analyzer( var resolvedGenerator: Generate = null val newProjectList = projectList.flatMap { - case AliasedGenerator(generator, names) if generator.childrenResolved => + + case AliasedGenerator(generator, names, outer) if generator.childrenResolved => // It's a sanity check, this should not happen as the previous case will throw // exception earlier. assert(resolvedGenerator == null, "More than one generator found in SELECT.") @@ -1653,7 +1671,7 @@ class Analyzer( Generate( generator, join = projectList.size > 1, // Only join if there are other expressions in SELECT. - outer = false, + outer = outer, qualifier = None, generatorOutput = ResolveGenerate.makeGeneratorOutput(generator, names), child) @@ -2053,7 +2071,7 @@ class Analyzer( case p => p transformExpressionsUp { - case udf @ ScalaUDF(func, _, inputs, _) => + case udf @ ScalaUDF(func, _, inputs, _, _) => val parameterTypes = ScalaReflection.getParameterTypes(func) assert(parameterTypes.length == inputs.length) @@ -2296,6 +2314,18 @@ class Analyzer( } } } + + /** + * Replace [[TimeZoneAwareExpression]] without [[TimeZone]] by its copy with session local + * time zone. + */ + object ResolveTimeZone extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressions { + case e: TimeZoneAwareExpression if e.timeZoneId.isEmpty => + e.withTimeZone(conf.sessionLocalTimeZone) + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index aa77a6efef347..f13a1f6d5d2f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -117,66 +117,72 @@ trait CheckAnalysis extends PredicateHelper { failAnalysis(s"Window specification $s is not valid because $m") case None => w } - case s @ ScalarSubquery(query, conditions, _) + + case s @ ScalarSubquery(query, conditions, _) => // If no correlation, the output must be exactly one column - if (conditions.isEmpty && query.output.size != 1) => + if (conditions.isEmpty && query.output.size != 1) { failAnalysis( s"Scalar subquery must return only one column, but got ${query.output.size}") + } + else if (conditions.nonEmpty) { + // Collect the columns from the subquery for further checking. + var subqueryColumns = conditions.flatMap(_.references).filter(query.output.contains) + + def checkAggregate(agg: Aggregate): Unit = { + // Make sure correlated scalar subqueries contain one row for every outer row by + // enforcing that they are aggregates containing exactly one aggregate expression. + // The analyzer has already checked that subquery contained only one output column, + // and added all the grouping expressions to the aggregate. + val aggregates = agg.expressions.flatMap(_.collect { + case a: AggregateExpression => a + }) + if (aggregates.isEmpty) { + failAnalysis("The output of a correlated scalar subquery must be aggregated") + } - case s @ ScalarSubquery(query, conditions, _) if conditions.nonEmpty => - - // Collect the columns from the subquery for further checking. - var subqueryColumns = conditions.flatMap(_.references).filter(query.output.contains) - - def checkAggregate(agg: Aggregate): Unit = { - // Make sure correlated scalar subqueries contain one row for every outer row by - // enforcing that they are aggregates which contain exactly one aggregate expressions. - // The analyzer has already checked that subquery contained only one output column, - // and added all the grouping expressions to the aggregate. - val aggregates = agg.expressions.flatMap(_.collect { - case a: AggregateExpression => a - }) - if (aggregates.isEmpty) { - failAnalysis("The output of a correlated scalar subquery must be aggregated") + // SPARK-18504/SPARK-18814: Block cases where GROUP BY columns + // are not part of the correlated columns. + val groupByCols = AttributeSet(agg.groupingExpressions.flatMap(_.references)) + val correlatedCols = AttributeSet(subqueryColumns) + val invalidCols = groupByCols -- correlatedCols + // GROUP BY columns must be a subset of columns in the predicates + if (invalidCols.nonEmpty) { + failAnalysis( + "A GROUP BY clause in a scalar correlated subquery " + + "cannot contain non-correlated columns: " + + invalidCols.mkString(",")) + } } - // SPARK-18504/SPARK-18814: Block cases where GROUP BY columns - // are not part of the correlated columns. - val groupByCols = AttributeSet(agg.groupingExpressions.flatMap(_.references)) - val correlatedCols = AttributeSet(subqueryColumns) - val invalidCols = groupByCols -- correlatedCols - // GROUP BY columns must be a subset of columns in the predicates - if (invalidCols.nonEmpty) { - failAnalysis( - "A GROUP BY clause in a scalar correlated subquery " + - "cannot contain non-correlated columns: " + - invalidCols.mkString(",")) - } - } + // Skip subquery aliases added by the Analyzer and the SQLBuilder. + // For projects, do the necessary mapping and skip to its child. + def cleanQuery(p: LogicalPlan): LogicalPlan = p match { + case s: SubqueryAlias => cleanQuery(s.child) + case p: Project => + // SPARK-18814: Map any aliases to their AttributeReference children + // for the checking in the Aggregate operators below this Project. + subqueryColumns = subqueryColumns.map { + xs => p.projectList.collectFirst { + case e @ Alias(child : AttributeReference, _) if e.exprId == xs.exprId => + child + }.getOrElse(xs) + } - // Skip subquery aliases added by the Analyzer and the SQLBuilder. - // For projects, do the necessary mapping and skip to its child. - def cleanQuery(p: LogicalPlan): LogicalPlan = p match { - case s: SubqueryAlias => cleanQuery(s.child) - case p: Project => - // SPARK-18814: Map any aliases to their AttributeReference children - // for the checking in the Aggregate operators below this Project. - subqueryColumns = subqueryColumns.map { - xs => p.projectList.collectFirst { - case e @ Alias(child : AttributeReference, _) if e.exprId == xs.exprId => - child - }.getOrElse(xs) - } + cleanQuery(p.child) + case child => child + } - cleanQuery(p.child) - case child => child + cleanQuery(query) match { + case a: Aggregate => checkAggregate(a) + case Filter(_, a: Aggregate) => checkAggregate(a) + case fail => failAnalysis(s"Correlated scalar subqueries must be Aggregated: $fail") + } } + checkAnalysis(query) + s - cleanQuery(query) match { - case a: Aggregate => checkAggregate(a) - case Filter(_, a: Aggregate) => checkAggregate(a) - case fail => failAnalysis(s"Correlated scalar subqueries must be Aggregated: $fail") - } + case s: SubqueryExpression => + checkAnalysis(s.plan) s } @@ -370,22 +376,6 @@ trait CheckAnalysis extends PredicateHelper { |Conflicting attributes: ${conflictingAttributes.mkString(",")} """.stripMargin) - case s: SimpleCatalogRelation => - failAnalysis( - s""" - |Hive support is required to select over the following tables: - |${s.catalogTable.identifier} - """.stripMargin) - - // TODO: We need to consolidate this kind of checks for InsertIntoTable - // with the rule of PreWriteCheck defined in extendedCheckRules. - case InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) => - failAnalysis( - s""" - |Hive support is required to insert into the following tables: - |${s.catalogTable.identifier} - """.stripMargin) - case InsertIntoTable(t, _, _, _, _) if !t.isInstanceOf[LeafNode] || t.isInstanceOf[Range] || diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 2b214c3c9d93e..eea3740be8a90 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -163,9 +163,11 @@ object FunctionRegistry { expression[Abs]("abs"), expression[Coalesce]("coalesce"), expression[Explode]("explode"), + expressionGeneratorOuter[Explode]("explode_outer"), expression[Greatest]("greatest"), expression[If]("if"), expression[Inline]("inline"), + expressionGeneratorOuter[Inline]("inline_outer"), expression[IsNaN]("isnan"), expression[IfNull]("ifnull"), expression[IsNull]("isnull"), @@ -176,6 +178,7 @@ object FunctionRegistry { expression[Nvl]("nvl"), expression[Nvl2]("nvl2"), expression[PosExplode]("posexplode"), + expressionGeneratorOuter[PosExplode]("posexplode_outer"), expression[Rand]("rand"), expression[Randn]("randn"), expression[Stack]("stack"), @@ -508,4 +511,13 @@ object FunctionRegistry { new ExpressionInfo(clazz.getCanonicalName, name) } } + + private def expressionGeneratorOuter[T <: Generator : ClassTag](name: String) + : (String, (ExpressionInfo, FunctionBuilder)) = { + val (_, (info, generatorBuilder)) = expression[T](name) + val outerBuilder = (args: Seq[Expression]) => { + GeneratorOuter(generatorBuilder(args).asInstanceOf[Generator]) + } + (name, (info, outerBuilder)) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 5f72fa8536e61..4177c2b03847d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -338,10 +338,13 @@ object TypeCoercion { case p @ BinaryComparison(left @ NullType(), right @ StringType()) => p.makeCopy(Array(Literal.create(null, StringType), right)) - case p @ BinaryComparison(left @ StringType(), right) if right.dataType != StringType => - p.makeCopy(Array(Cast(left, DoubleType), right)) - case p @ BinaryComparison(left, right @ StringType()) if left.dataType != StringType => - p.makeCopy(Array(left, Cast(right, DoubleType))) + // When compare string with atomic type, case string to that type. + case p @ BinaryComparison(left @ StringType(), right @ AtomicType()) + if right.dataType != StringType => + p.makeCopy(Array(Cast(left, right.dataType), right)) + case p @ BinaryComparison(left @ AtomicType(), right @ StringType()) + if left.dataType != StringType => + p.makeCopy(Array(left, Cast(right, left.dataType))) case i @ In(a @ DateType(), b) if b.forall(_.dataType == StringType) => i.makeCopy(Array(Cast(a, StringType), b)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index c2666b2ab9129..f4d016cb96711 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -87,7 +87,7 @@ object UnsupportedOperationChecker { * data. */ def containsCompleteData(subplan: LogicalPlan): Boolean = { - val aggs = plan.collect { case a@Aggregate(_, _, _) if a.isStreaming => a } + val aggs = subplan.collect { case a@Aggregate(_, _, _) if a.isStreaming => a } // Either the subplan has no streaming source, or it has aggregation with Complete mode !subplan.isStreaming || (aggs.nonEmpty && outputMode == InternalOutputModes.Complete) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 12af9e0c32611..e9543f79878b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -21,13 +21,13 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable +import com.google.common.cache.{Cache, CacheBuilder} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} @@ -117,6 +117,14 @@ class SessionCatalog( if (conf.caseSensitiveAnalysis) name else name.toLowerCase } + /** + * A cache of qualified table name to table relation plan. + */ + val tableRelationCache: Cache[QualifiedTableName, LogicalPlan] = { + // TODO: create a config instead of hardcode 1000 here. + CacheBuilder.newBuilder().maximumSize(1000).build[QualifiedTableName, LogicalPlan]() + } + /** * This method is used to make the given path qualified before we * store this path in the underlying external catalog. So, when a path @@ -331,7 +339,7 @@ class SessionCatalog( def loadPartition( name: TableIdentifier, loadPath: String, - partition: TablePartitionSpec, + spec: TablePartitionSpec, isOverwrite: Boolean, holdDDLTime: Boolean, inheritTableSpecs: Boolean, @@ -340,8 +348,9 @@ class SessionCatalog( val table = formatTableName(name.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Some(db))) + requireNonEmptyValueInPartitionSpec(Seq(spec)) externalCatalog.loadPartition( - db, table, loadPath, partition, isOverwrite, holdDDLTime, inheritTableSpecs, isSrcLocal) + db, table, loadPath, spec, isOverwrite, holdDDLTime, inheritTableSpecs, isSrcLocal) } def defaultTablePath(tableIdent: TableIdentifier): String = { @@ -572,7 +581,7 @@ class SessionCatalog( val relationAlias = alias.getOrElse(table) if (db == globalTempViewManager.database) { globalTempViewManager.get(table).map { viewDef => - SubqueryAlias(relationAlias, viewDef, Some(name)) + SubqueryAlias(relationAlias, viewDef, None) }.getOrElse(throw new NoSuchTableException(db, table)) } else if (name.database.isDefined || !tempTables.contains(table)) { val metadata = externalCatalog.getTable(db, table) @@ -585,12 +594,12 @@ class SessionCatalog( desc = metadata, output = metadata.schema.toAttributes, child = parser.parsePlan(viewText)) - SubqueryAlias(relationAlias, child, Option(name)) + SubqueryAlias(relationAlias, child, Some(name.copy(table = table, database = Some(db)))) } else { SubqueryAlias(relationAlias, SimpleCatalogRelation(metadata), None) } } else { - SubqueryAlias(relationAlias, tempTables(table), Option(name)) + SubqueryAlias(relationAlias, tempTables(table), None) } } } @@ -650,14 +659,21 @@ class SessionCatalog( * Refresh the cache entry for a metastore table, if any. */ def refreshTable(name: TableIdentifier): Unit = synchronized { + val dbName = formatDatabaseName(name.database.getOrElse(currentDb)) + val tableName = formatTableName(name.table) + // Go through temporary tables and invalidate them. - // If the database is defined, this is definitely not a temp table. + // If the database is defined, this may be a global temporary view. // If the database is not defined, there is a good chance this is a temp table. if (name.database.isEmpty) { - tempTables.get(formatTableName(name.table)).foreach(_.refresh()) - } else if (formatDatabaseName(name.database.get) == globalTempViewManager.database) { - globalTempViewManager.get(formatTableName(name.table)).foreach(_.refresh()) + tempTables.get(tableName).foreach(_.refresh()) + } else if (dbName == globalTempViewManager.database) { + globalTempViewManager.get(tableName).foreach(_.refresh()) } + + // Also invalidate the table relation cache. + val qualifiedTableName = QualifiedTableName(dbName, tableName) + tableRelationCache.invalidate(qualifiedTableName) } /** @@ -693,6 +709,7 @@ class SessionCatalog( requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) + requireNonEmptyValueInPartitionSpec(parts.map(_.spec)) externalCatalog.createPartitions(db, table, parts, ignoreIfExists) } @@ -711,6 +728,7 @@ class SessionCatalog( requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) requirePartialMatchedPartitionSpec(specs, getTableMetadata(tableName)) + requireNonEmptyValueInPartitionSpec(specs) externalCatalog.dropPartitions(db, table, specs, ignoreIfNotExists, purge, retainData) } @@ -731,6 +749,8 @@ class SessionCatalog( requireTableExists(TableIdentifier(table, Option(db))) requireExactMatchedPartitionSpec(specs, tableMetadata) requireExactMatchedPartitionSpec(newSpecs, tableMetadata) + requireNonEmptyValueInPartitionSpec(specs) + requireNonEmptyValueInPartitionSpec(newSpecs) externalCatalog.renamePartitions(db, table, specs, newSpecs) } @@ -749,6 +769,7 @@ class SessionCatalog( requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) + requireNonEmptyValueInPartitionSpec(parts.map(_.spec)) externalCatalog.alterPartitions(db, table, parts) } @@ -762,6 +783,7 @@ class SessionCatalog( requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) requireExactMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName)) + requireNonEmptyValueInPartitionSpec(Seq(spec)) externalCatalog.getPartition(db, table, spec) } @@ -781,6 +803,7 @@ class SessionCatalog( requireTableExists(TableIdentifier(table, Option(db))) partialSpec.foreach { spec => requirePartialMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName)) + requireNonEmptyValueInPartitionSpec(Seq(spec)) } externalCatalog.listPartitionNames(db, table, partialSpec) } @@ -801,6 +824,7 @@ class SessionCatalog( requireTableExists(TableIdentifier(table, Option(db))) partialSpec.foreach { spec => requirePartialMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName)) + requireNonEmptyValueInPartitionSpec(Seq(spec)) } externalCatalog.listPartitions(db, table, partialSpec) } @@ -819,6 +843,19 @@ class SessionCatalog( externalCatalog.listPartitionsByFilter(db, table, predicates) } + /** + * Verify if the input partition spec has any empty value. + */ + private def requireNonEmptyValueInPartitionSpec(specs: Seq[TablePartitionSpec]): Unit = { + specs.foreach { s => + if (s.values.exists(_.isEmpty)) { + val spec = s.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]") + throw new AnalysisException( + s"Partition spec is invalid. The spec ($spec) contains an empty partition column value") + } + } + } + /** * Verify if the input partition spec exactly matches the existing defined partition spec * The columns must be the same but the orders could be different. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 2adccdd7bf61d..a8fa78d41cb57 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -26,10 +26,10 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIden import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Cast, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types.StructType - /** * A function defined in the catalog. * @@ -114,7 +114,9 @@ case class CatalogTablePartition( */ def toRow(partitionSchema: StructType): InternalRow = { InternalRow.fromSeq(partitionSchema.map { field => - Cast(Literal(spec(field.name)), field.dataType).eval() + // TODO: use correct timezone for partition values. + Cast(Literal(spec(field.name)), field.dataType, + Option(DateTimeUtils.defaultTimeZone().getID)).eval() }) } } @@ -175,7 +177,6 @@ case class CatalogTable( lastAccessTime: Long = -1, properties: Map[String, String] = Map.empty, stats: Option[CatalogStatistics] = None, - viewOriginalText: Option[String] = None, viewText: Option[String] = None, comment: Option[String] = None, unsupportedFeatures: Seq[String] = Seq.empty, @@ -183,10 +184,15 @@ case class CatalogTable( import CatalogTable._ - /** schema of this table's partition columns */ - def partitionSchema: StructType = StructType(schema.filter { - c => partitionColumnNames.contains(c.name) - }) + /** + * schema of this table's partition columns + */ + def partitionSchema: StructType = { + val partitionFields = schema.takeRight(partitionColumnNames.length) + assert(partitionFields.map(_.name) == partitionColumnNames) + + StructType(partitionFields) + } /** Return the database this table was specified to belong to, assuming it exists. */ def database: String = identifier.database.getOrElse { @@ -223,25 +229,6 @@ case class CatalogTable( ) } - /** - * Insert/Update the view query output column names in `properties`. - */ - def withQueryColumnNames(columns: Seq[String]): CatalogTable = { - val props = new mutable.HashMap[String, String] - if (columns.nonEmpty) { - props.put(VIEW_QUERY_OUTPUT_NUM_COLUMNS, columns.length.toString) - columns.zipWithIndex.foreach { case (colName, index) => - props.put(s"$VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX$index", colName) - } - } - - // We can't use `filterKeys` here, as the map returned by `filterKeys` is not serializable, - // while `CatalogTable` should be serializable. - copy(properties = properties.filterNot { case (key, _) => - key.startsWith(VIEW_QUERY_OUTPUT_PREFIX) - } ++ props) - } - /** Syntactic sugar to update a field in `storage`. */ def withNewStorage( locationUri: Option[String] = storage.locationUri, @@ -280,7 +267,6 @@ case class CatalogTable( if (provider.isDefined) s"Provider: ${provider.get}" else "", if (partitionColumnNames.nonEmpty) s"Partition Columns: $partitionColumns" else "" ) ++ bucketStrings ++ Seq( - viewOriginalText.map("Original View: " + _).getOrElse(""), viewText.map("View: " + _).getOrElse(""), comment.map("Comment: " + _).getOrElse(""), if (properties.nonEmpty) s"Properties: $tableProperties" else "", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala index e876450c73fde..65e497afc12cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala @@ -37,7 +37,7 @@ object Canonicalize extends { } /** Remove names and nullability from types. */ - private def ignoreNamesTypes(e: Expression): Expression = e match { + private[expressions] def ignoreNamesTypes(e: Expression): Expression = e match { case a: AttributeReference => AttributeReference("none", a.dataType.asNullable)(exprId = a.exprId) case _ => e @@ -78,13 +78,11 @@ object Canonicalize extends { case GreaterThanOrEqual(l, r) if l.hashCode() > r.hashCode() => LessThanOrEqual(r, l) case LessThanOrEqual(l, r) if l.hashCode() > r.hashCode() => GreaterThanOrEqual(r, l) - case Not(GreaterThan(l, r)) if l.hashCode() > r.hashCode() => GreaterThan(r, l) + // Note in the following `NOT` cases, `l.hashCode() <= r.hashCode()` holds. The reason is that + // canonicalization is conducted bottom-up -- see [[Expression.canonicalized]]. case Not(GreaterThan(l, r)) => LessThanOrEqual(l, r) - case Not(LessThan(l, r)) if l.hashCode() > r.hashCode() => LessThan(r, l) case Not(LessThan(l, r)) => GreaterThanOrEqual(l, r) - case Not(GreaterThanOrEqual(l, r)) if l.hashCode() > r.hashCode() => GreaterThanOrEqual(r, l) case Not(GreaterThanOrEqual(l, r)) => LessThan(l, r) - case Not(LessThanOrEqual(l, r)) if l.hashCode() > r.hashCode() => LessThanOrEqual(r, l) case Not(LessThanOrEqual(l, r)) => GreaterThan(l, r) case _ => e diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index ad59271e5b02c..a36d3507d92ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -131,7 +131,12 @@ object Cast { private def resolvableNullability(from: Boolean, to: Boolean) = !from || to } -/** Cast the child expression to the target data type. */ +/** + * Cast the child expression to the target data type. + * + * When cast from/to timezone related types, we need timeZoneId, which will be resolved with + * session local timezone by an analyzer [[ResolveTimeZone]]. + */ @ExpressionDescription( usage = "_FUNC_(expr AS type) - Casts the value `expr` to the target data type `type`.", extended = """ @@ -139,7 +144,10 @@ object Cast { > SELECT _FUNC_('10' as int); 10 """) -case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with NullIntolerant { +case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String] = None) + extends UnaryExpression with TimeZoneAwareExpression with NullIntolerant { + + def this(child: Expression, dataType: DataType) = this(child, dataType, None) override def toString: String = s"cast($child as ${dataType.simpleString})" @@ -154,6 +162,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w override def nullable: Boolean = Cast.forceNullable(child.dataType, dataType) || child.nullable + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + // [[func]] assumes the input is no longer null because eval already does the null check. @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) @@ -162,7 +173,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes) case DateType => buildCast[Int](_, d => UTF8String.fromString(DateTimeUtils.dateToString(d))) case TimestampType => buildCast[Long](_, - t => UTF8String.fromString(DateTimeUtils.timestampToString(t))) + t => UTF8String.fromString(DateTimeUtils.timestampToString(t, timeZone))) case _ => buildCast[Any](_, o => UTF8String.fromString(o.toString)) } @@ -207,7 +218,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // TimestampConverter private[this] def castToTimestamp(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, utfs => DateTimeUtils.stringToTimestamp(utfs).orNull) + buildCast[UTF8String](_, utfs => DateTimeUtils.stringToTimestamp(utfs, timeZone).orNull) case BooleanType => buildCast[Boolean](_, b => if (b) 1L else 0) case LongType => @@ -219,7 +230,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case ByteType => buildCast[Byte](_, b => longToTimestamp(b.toLong)) case DateType => - buildCast[Int](_, d => DateTimeUtils.daysToMillis(d) * 1000) + buildCast[Int](_, d => DateTimeUtils.daysToMillis(d, timeZone) * 1000) // TimestampWritable.decimalToTimestamp case DecimalType() => buildCast[Decimal](_, d => decimalToTimestamp(d)) @@ -254,7 +265,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. - buildCast[Long](_, t => DateTimeUtils.millisToDays(t / 1000L)) + buildCast[Long](_, t => DateTimeUtils.millisToDays(t / 1000L, timeZone)) } // IntervalConverter @@ -531,8 +542,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString( org.apache.spark.sql.catalyst.util.DateTimeUtils.dateToString($c));""" case TimestampType => + val tz = ctx.addReferenceMinorObj(timeZone) (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString( - org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c));""" + org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));""" case _ => (c, evPrim, evNull) => s"$evPrim = UTF8String.fromString(String.valueOf($c));" } @@ -558,8 +570,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } """ case TimestampType => + val tz = ctx.addReferenceMinorObj(timeZone) (c, evPrim, evNull) => - s"$evPrim = org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToDays($c / 1000L);"; + s"$evPrim = org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToDays($c / 1000L, $tz);" case _ => (c, evPrim, evNull) => s"$evNull = true;" } @@ -637,11 +650,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w from: DataType, ctx: CodegenContext): CastFunction = from match { case StringType => + val tz = ctx.addReferenceMinorObj(timeZone) val longOpt = ctx.freshName("longOpt") (c, evPrim, evNull) => s""" scala.Option $longOpt = - org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToTimestamp($c); + org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToTimestamp($c, $tz); if ($longOpt.isDefined()) { $evPrim = ((Long) $longOpt.get()).longValue(); } else { @@ -653,8 +667,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case _: IntegralType => (c, evPrim, evNull) => s"$evPrim = ${longToTimeStampCode(c)};" case DateType => + val tz = ctx.addReferenceMinorObj(timeZone) (c, evPrim, evNull) => - s"$evPrim = org.apache.spark.sql.catalyst.util.DateTimeUtils.daysToMillis($c) * 1000;" + s"$evPrim = org.apache.spark.sql.catalyst.util.DateTimeUtils.daysToMillis($c, $tz) * 1000;" case DecimalType() => (c, evPrim, evNull) => s"$evPrim = ${decimalToTimestampCode(c)};" case DoubleType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala index 6c246a5663ca3..f8644c2cd672c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala @@ -67,28 +67,34 @@ class EquivalentExpressions { /** * Adds the expression to this data structure recursively. Stops if a matching expression * is found. That is, if `expr` has already been added, its children are not added. - * If ignoreLeaf is true, leaf nodes are ignored. */ - def addExprTree( - root: Expression, - ignoreLeaf: Boolean = true, - skipReferenceToExpressions: Boolean = true): Unit = { - val skip = (root.isInstanceOf[LeafExpression] && ignoreLeaf) || + def addExprTree(expr: Expression): Unit = { + val skip = expr.isInstanceOf[LeafExpression] || // `LambdaVariable` is usually used as a loop variable, which can't be evaluated ahead of the // loop. So we can't evaluate sub-expressions containing `LambdaVariable` at the beginning. - root.find(_.isInstanceOf[LambdaVariable]).isDefined - // There are some special expressions that we should not recurse into children. + expr.find(_.isInstanceOf[LambdaVariable]).isDefined + + // There are some special expressions that we should not recurse into all of its children. // 1. CodegenFallback: it's children will not be used to generate code (call eval() instead) - // 2. ReferenceToExpressions: it's kind of an explicit sub-expression elimination. - val shouldRecurse = root match { - // TODO: some expressions implements `CodegenFallback` but can still do codegen, - // e.g. `CaseWhen`, we should support them. - case _: CodegenFallback => false - case _: ReferenceToExpressions if skipReferenceToExpressions => false - case _ => true + // 2. If: common subexpressions will always be evaluated at the beginning, but the true and + // false expressions in `If` may not get accessed, according to the predicate + // expression. We should only recurse into the predicate expression. + // 3. CaseWhen: like `If`, the children of `CaseWhen` only get accessed in a certain + // condition. We should only recurse into the first condition expression as it + // will always get accessed. + // 4. Coalesce: it's also a conditional expression, we should only recurse into the first + // children, because others may not get accessed. + def childrenToRecurse: Seq[Expression] = expr match { + case _: CodegenFallback => Nil + case i: If => i.predicate :: Nil + // `CaseWhen` implements `CodegenFallback`, we only need to handle `CaseWhenCodegen` here. + case c: CaseWhenCodegen => c.children.head :: Nil + case c: Coalesce => c.children.head :: Nil + case other => other.children } - if (!skip && !addExpr(root) && shouldRecurse) { - root.children.foreach(addExprTree(_, ignoreLeaf)) + + if (!skip && !addExpr(expr)) { + childrenToRecurse.foreach(addExprTree) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala index 32358a99e7ce7..84027b53dca27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.types.{DataType, LongType} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 476e37e6a9bac..7c57025f995d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -117,7 +117,7 @@ object UnsafeProjection { * Returns an UnsafeProjection for given Array of DataTypes. */ def create(fields: Array[DataType]): UnsafeProjection = { - create(fields.zipWithIndex.map(x => new BoundReference(x._2, x._1, true))) + create(fields.zipWithIndex.map(x => BoundReference(x._2, x._1, true))) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala deleted file mode 100644 index 2ca77e8394e17..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.expressions.objects.LambdaVariable -import org.apache.spark.sql.types.DataType - -/** - * A special expression that evaluates [[BoundReference]]s by given expressions instead of the - * input row. - * - * @param result The expression that contains [[BoundReference]] and produces the final output. - * @param children The expressions that used as input values for [[BoundReference]]. - */ -case class ReferenceToExpressions(result: Expression, children: Seq[Expression]) - extends Expression { - - override def nullable: Boolean = result.nullable - override def dataType: DataType = result.dataType - - override def checkInputDataTypes(): TypeCheckResult = { - if (result.references.nonEmpty) { - return TypeCheckFailure("The result expression cannot reference to any attributes.") - } - - var maxOrdinal = -1 - result foreach { - case b: BoundReference if b.ordinal > maxOrdinal => maxOrdinal = b.ordinal - case _ => - } - if (maxOrdinal > children.length) { - return TypeCheckFailure(s"The result expression need $maxOrdinal input expressions, but " + - s"there are only ${children.length} inputs.") - } - - TypeCheckSuccess - } - - private lazy val projection = UnsafeProjection.create(children) - - override def eval(input: InternalRow): Any = { - result.eval(projection(input)) - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val childrenGen = children.map(_.genCode(ctx)) - val (classChildrenVars, initClassChildrenVars) = childrenGen.zip(children).map { - case (childGen, child) => - // SPARK-18125: The children vars are local variables. If the result expression uses - // splitExpression, those variables cannot be accessed so compilation fails. - // To fix it, we use class variables to hold those local variables. - val classChildVarName = ctx.freshName("classChildVar") - val classChildVarIsNull = ctx.freshName("classChildVarIsNull") - ctx.addMutableState(ctx.javaType(child.dataType), classChildVarName, "") - ctx.addMutableState("boolean", classChildVarIsNull, "") - - val classChildVar = - LambdaVariable(classChildVarName, classChildVarIsNull, child.dataType, child.nullable) - - val initCode = s"${classChildVar.value} = ${childGen.value};\n" + - s"${classChildVar.isNull} = ${childGen.isNull};" - - (classChildVar, initCode) - }.unzip - - val resultGen = result.transform { - case b: BoundReference => classChildrenVars(b.ordinal) - }.genCode(ctx) - - ExprCode(code = childrenGen.map(_.code).mkString("\n") + initClassChildrenVars.mkString("\n") + - resultGen.code, isNull = resultGen.isNull, value = resultGen.value) - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 6cfdea9fdf9c5..228f4b756c8b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -35,17 +35,20 @@ import org.apache.spark.sql.types.DataType * not want to perform coercion, simply use "Nil". Note that it would've been * better to use Option of Seq[DataType] so we can use "None" as the case for no * type coercion. However, that would require more refactoring of the codebase. + * @param udfName The user-specified name of this UDF. */ case class ScalaUDF( function: AnyRef, dataType: DataType, children: Seq[Expression], - inputTypes: Seq[DataType] = Nil) + inputTypes: Seq[DataType] = Nil, + udfName: Option[String] = None) extends Expression with ImplicitCastInputTypes with NonSQLExpression { override def nullable: Boolean = true - override def toString: String = s"UDF(${children.mkString(", ")})" + override def toString: String = + s"${udfName.map(name => s"UDF:$name").getOrElse("UDF")}(${children.mkString(", ")})" // scalastyle:off line.size.limit diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index 18b7f95fea65b..db062f1a543fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -21,7 +21,6 @@ import java.nio.ByteBuffer import com.google.common.primitives.{Doubles, Ints, Longs} -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala index eaeb010b0e4fa..5b4ce47fd53ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} import java.util -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 411f058510ca7..26cd9ab665383 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} - import scala.collection.generic.Growable import scala.collection.mutable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index f8f868b59b967..04b812e79ee5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -726,7 +726,7 @@ class CodegenContext { val subExprEliminationExprs = mutable.HashMap.empty[Expression, SubExprEliminationState] // Add each expression tree and compute the common subexpressions. - expressions.foreach(equivalentExpressions.addExprTree(_, true, false)) + expressions.foreach(equivalentExpressions.addExprTree) // Get all the expressions that appear at least twice and set up the state for subexpression // elimination. @@ -734,10 +734,10 @@ class CodegenContext { val codes = commonExprs.map { e => val expr = e.head // Generate the code for this expression tree. - val code = expr.genCode(this) - val state = SubExprEliminationState(code.isNull, code.value) + val eval = expr.genCode(this) + val state = SubExprEliminationState(eval.isNull, eval.value) e.foreach(subExprEliminationExprs.put(_, state)) - code.code.trim + eval.code.trim } SubExprCodes(codes, subExprEliminationExprs.toMap) } @@ -747,7 +747,7 @@ class CodegenContext { * common subexpressions, generates the functions that evaluate those expressions and populates * the mapping of common subexpressions to the generated functions. */ - private def subexpressionElimination(expressions: Seq[Expression]) = { + private def subexpressionElimination(expressions: Seq[Expression]): Unit = { // Add each expression tree and compute the common subexpressions. expressions.foreach(equivalentExpressions.addExprTree(_)) @@ -761,13 +761,13 @@ class CodegenContext { val value = s"${fnName}Value" // Generate the code for this expression tree and wrap it in a function. - val code = expr.genCode(this) + val eval = expr.genCode(this) val fn = s""" |private void $fnName(InternalRow $INPUT_ROW) { - | ${code.code.trim} - | $isNull = ${code.isNull}; - | $value = ${code.value}; + | ${eval.code.trim} + | $isNull = ${eval.isNull}; + | $value = ${eval.value}; |} """.stripMargin @@ -780,9 +780,6 @@ class CodegenContext { // The cost of doing subexpression elimination is: // 1. Extra function call, although this is probably *good* as the JIT can decide to // inline or not. - // 2. Extra branch to check isLoaded. This branch is likely to be predicted correctly - // very often. The reason it is not loaded is because of a prior branch. - // 3. Extra store into isLoaded. // The benefit doing subexpression elimination is: // 1. Running the expression logic. Even for a simple expression, it is likely more than 3 // above. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index ef1ac360daada..bad8a7123017b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import java.text.SimpleDateFormat -import java.util.{Calendar, Locale, TimeZone} +import java.text.DateFormat +import java.util.{Calendar, TimeZone} -import scala.util.Try +import scala.util.control.NonFatal import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodegenFallback, ExprCode} @@ -29,6 +29,20 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +/** + * Common base class for time zone aware expressions. + */ +trait TimeZoneAwareExpression extends Expression { + + /** the timezone ID to be used to evaluate value. */ + def timeZoneId: Option[String] + + /** Returns a copy of this expression with the specified timeZoneId. */ + def withTimeZone(timeZoneId: String): TimeZoneAwareExpression + + @transient lazy val timeZone: TimeZone = TimeZone.getTimeZone(timeZoneId.get) +} + /** * Returns the current date at the start of query evaluation. * All calls of current_date within the same query return the same value. @@ -37,14 +51,21 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} */ @ExpressionDescription( usage = "_FUNC_() - Returns the current date at the start of query evaluation.") -case class CurrentDate() extends LeafExpression with CodegenFallback { +case class CurrentDate(timeZoneId: Option[String] = None) + extends LeafExpression with TimeZoneAwareExpression with CodegenFallback { + + def this() = this(None) + override def foldable: Boolean = true override def nullable: Boolean = false override def dataType: DataType = DateType + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + override def eval(input: InternalRow): Any = { - DateTimeUtils.millisToDays(System.currentTimeMillis()) + DateTimeUtils.millisToDays(System.currentTimeMillis(), timeZone) } override def prettyName: String = "current_date" @@ -78,11 +99,19 @@ case class CurrentTimestamp() extends LeafExpression with CodegenFallback { * * There is no code generation since this expression should be replaced with a literal. */ -case class CurrentBatchTimestamp(timestampMs: Long, dataType: DataType) - extends LeafExpression with Nondeterministic with CodegenFallback { +case class CurrentBatchTimestamp( + timestampMs: Long, + dataType: DataType, + timeZoneId: Option[String] = None) + extends LeafExpression with TimeZoneAwareExpression with Nondeterministic with CodegenFallback { + + def this(timestampMs: Long, dataType: DataType) = this(timestampMs, dataType, None) override def nullable: Boolean = false + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + override def prettyName: String = "current_batch_timestamp" override protected def initializeInternal(partitionIndex: Int): Unit = {} @@ -96,7 +125,7 @@ case class CurrentBatchTimestamp(timestampMs: Long, dataType: DataType) def toLiteral: Literal = dataType match { case _: TimestampType => Literal(DateTimeUtils.fromJavaTimestamp(new Timestamp(timestampMs)), TimestampType) - case _: DateType => Literal(DateTimeUtils.millisToDays(timestampMs), DateType) + case _: DateType => Literal(DateTimeUtils.millisToDays(timestampMs, timeZone), DateType) } } @@ -172,19 +201,26 @@ case class DateSub(startDate: Expression, days: Expression) > SELECT _FUNC_('2009-07-30 12:58:59'); 12 """) -case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { +case class Hour(child: Expression, timeZoneId: Option[String] = None) + extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + + def this(child: Expression) = this(child, None) override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) override def dataType: DataType = IntegerType + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + override protected def nullSafeEval(timestamp: Any): Any = { - DateTimeUtils.getHours(timestamp.asInstanceOf[Long]) + DateTimeUtils.getHours(timestamp.asInstanceOf[Long], timeZone) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val tz = ctx.addReferenceMinorObj(timeZone) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getHours($c)") + defineCodeGen(ctx, ev, c => s"$dtu.getHours($c, $tz)") } } @@ -195,19 +231,26 @@ case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInpu > SELECT _FUNC_('2009-07-30 12:58:59'); 58 """) -case class Minute(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { +case class Minute(child: Expression, timeZoneId: Option[String] = None) + extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + + def this(child: Expression) = this(child, None) override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) override def dataType: DataType = IntegerType + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + override protected def nullSafeEval(timestamp: Any): Any = { - DateTimeUtils.getMinutes(timestamp.asInstanceOf[Long]) + DateTimeUtils.getMinutes(timestamp.asInstanceOf[Long], timeZone) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val tz = ctx.addReferenceMinorObj(timeZone) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getMinutes($c)") + defineCodeGen(ctx, ev, c => s"$dtu.getMinutes($c, $tz)") } } @@ -218,19 +261,26 @@ case class Minute(child: Expression) extends UnaryExpression with ImplicitCastIn > SELECT _FUNC_('2009-07-30 12:58:59'); 59 """) -case class Second(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { +case class Second(child: Expression, timeZoneId: Option[String] = None) + extends UnaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + + def this(child: Expression) = this(child, None) override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) override def dataType: DataType = IntegerType + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + override protected def nullSafeEval(timestamp: Any): Any = { - DateTimeUtils.getSeconds(timestamp.asInstanceOf[Long]) + DateTimeUtils.getSeconds(timestamp.asInstanceOf[Long], timeZone) } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val tz = ctx.addReferenceMinorObj(timeZone) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$dtu.getSeconds($c)") + defineCodeGen(ctx, ev, c => s"$dtu.getSeconds($c, $tz)") } } @@ -401,22 +451,28 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa 2016 """) // scalastyle:on line.size.limit -case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression - with ImplicitCastInputTypes { +case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Option[String] = None) + extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + + def this(left: Expression, right: Expression) = this(left, right, None) override def dataType: DataType = StringType override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + override protected def nullSafeEval(timestamp: Any, format: Any): Any = { - val sdf = new SimpleDateFormat(format.toString, Locale.US) - UTF8String.fromString(sdf.format(new java.util.Date(timestamp.asInstanceOf[Long] / 1000))) + val df = DateTimeUtils.newDateFormat(format.toString, timeZone) + UTF8String.fromString(df.format(new java.util.Date(timestamp.asInstanceOf[Long] / 1000))) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val sdf = classOf[SimpleDateFormat].getName + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + val tz = ctx.addReferenceMinorObj(timeZone) defineCodeGen(ctx, ev, (timestamp, format) => { - s"""UTF8String.fromString((new $sdf($format.toString())) + s"""UTF8String.fromString($dtu.newDateFormat($format.toString(), $tz) .format(new java.util.Date($timestamp / 1000)))""" }) } @@ -435,10 +491,20 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx > SELECT _FUNC_('2016-04-08', 'yyyy-MM-dd'); 1460041200 """) -case class ToUnixTimestamp(timeExp: Expression, format: Expression) extends UnixTime { +case class ToUnixTimestamp( + timeExp: Expression, + format: Expression, + timeZoneId: Option[String] = None) + extends UnixTime { + + def this(timeExp: Expression, format: Expression) = this(timeExp, format, None) + override def left: Expression = timeExp override def right: Expression = format + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + def this(time: Expression) = { this(time, Literal("yyyy-MM-dd HH:mm:ss")) } @@ -465,10 +531,17 @@ case class ToUnixTimestamp(timeExp: Expression, format: Expression) extends Unix > SELECT _FUNC_('2016-04-08', 'yyyy-MM-dd'); 1460041200 """) -case class UnixTimestamp(timeExp: Expression, format: Expression) extends UnixTime { +case class UnixTimestamp(timeExp: Expression, format: Expression, timeZoneId: Option[String] = None) + extends UnixTime { + + def this(timeExp: Expression, format: Expression) = this(timeExp, format, None) + override def left: Expression = timeExp override def right: Expression = format + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + def this(time: Expression) = { this(time, Literal("yyyy-MM-dd HH:mm:ss")) } @@ -480,7 +553,8 @@ case class UnixTimestamp(timeExp: Expression, format: Expression) extends UnixTi override def prettyName: String = "unix_timestamp" } -abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { +abstract class UnixTime + extends BinaryExpression with TimeZoneAwareExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(StringType, DateType, TimestampType), StringType) @@ -489,8 +563,12 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { override def nullable: Boolean = true private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] - private lazy val formatter: SimpleDateFormat = - Try(new SimpleDateFormat(constFormat.toString, Locale.US)).getOrElse(null) + private lazy val formatter: DateFormat = + try { + DateTimeUtils.newDateFormat(constFormat.toString, timeZone) + } catch { + case NonFatal(_) => null + } override def eval(input: InternalRow): Any = { val t = left.eval(input) @@ -499,15 +577,19 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { } else { left.dataType match { case DateType => - DateTimeUtils.daysToMillis(t.asInstanceOf[Int]) / 1000L + DateTimeUtils.daysToMillis(t.asInstanceOf[Int], timeZone) / 1000L case TimestampType => t.asInstanceOf[Long] / 1000000L case StringType if right.foldable => if (constFormat == null || formatter == null) { null } else { - Try(formatter.parse( - t.asInstanceOf[UTF8String].toString).getTime / 1000L).getOrElse(null) + try { + formatter.parse( + t.asInstanceOf[UTF8String].toString).getTime / 1000L + } catch { + case NonFatal(_) => null + } } case StringType => val f = right.eval(input) @@ -515,8 +597,12 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { null } else { val formatString = f.asInstanceOf[UTF8String].toString - Try(new SimpleDateFormat(formatString, Locale.US).parse( - t.asInstanceOf[UTF8String].toString).getTime / 1000L).getOrElse(null) + try { + DateTimeUtils.newDateFormat(formatString, timeZone).parse( + t.asInstanceOf[UTF8String].toString).getTime / 1000L + } catch { + case NonFatal(_) => null + } } } } @@ -525,11 +611,11 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { left.dataType match { case StringType if right.foldable => - val sdf = classOf[SimpleDateFormat].getName + val df = classOf[DateFormat].getName if (formatter == null) { ExprCode("", "true", ctx.defaultValue(dataType)) } else { - val formatterName = ctx.addReferenceObj("formatter", formatter, sdf) + val formatterName = ctx.addReferenceObj("formatter", formatter, df) val eval1 = left.genCode(ctx) ev.copy(code = s""" ${eval1.code} @@ -544,12 +630,13 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { }""") } case StringType => - val sdf = classOf[SimpleDateFormat].getName + val tz = ctx.addReferenceMinorObj(timeZone) + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (string, format) => { s""" try { - ${ev.value} = - (new $sdf($format.toString())).parse($string.toString()).getTime() / 1000L; + ${ev.value} = $dtu.newDateFormat($format.toString(), $tz) + .parse($string.toString()).getTime() / 1000L; } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; } catch (java.text.ParseException e) { @@ -567,6 +654,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { ${ev.value} = ${eval1.value} / 1000000L; }""") case DateType => + val tz = ctx.addReferenceMinorObj(timeZone) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") val eval1 = left.genCode(ctx) ev.copy(code = s""" @@ -574,7 +662,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { boolean ${ev.isNull} = ${eval1.isNull}; ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; if (!${ev.isNull}) { - ${ev.value} = $dtu.daysToMillis(${eval1.value}) / 1000L; + ${ev.value} = $dtu.daysToMillis(${eval1.value}, $tz) / 1000L; }""") } } @@ -593,8 +681,10 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { > SELECT _FUNC_(0, 'yyyy-MM-dd HH:mm:ss'); 1970-01-01 00:00:00 """) -case class FromUnixTime(sec: Expression, format: Expression) - extends BinaryExpression with ImplicitCastInputTypes { +case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[String] = None) + extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + + def this(sec: Expression, format: Expression) = this(sec, format, None) override def left: Expression = sec override def right: Expression = format @@ -610,9 +700,16 @@ case class FromUnixTime(sec: Expression, format: Expression) override def inputTypes: Seq[AbstractDataType] = Seq(LongType, StringType) + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] - private lazy val formatter: SimpleDateFormat = - Try(new SimpleDateFormat(constFormat.toString, Locale.US)).getOrElse(null) + private lazy val formatter: DateFormat = + try { + DateTimeUtils.newDateFormat(constFormat.toString, timeZone) + } catch { + case NonFatal(_) => null + } override def eval(input: InternalRow): Any = { val time = left.eval(input) @@ -623,30 +720,36 @@ case class FromUnixTime(sec: Expression, format: Expression) if (constFormat == null || formatter == null) { null } else { - Try(UTF8String.fromString(formatter.format( - new java.util.Date(time.asInstanceOf[Long] * 1000L)))).getOrElse(null) + try { + UTF8String.fromString(formatter.format( + new java.util.Date(time.asInstanceOf[Long] * 1000L))) + } catch { + case NonFatal(_) => null + } } } else { val f = format.eval(input) if (f == null) { null } else { - Try( - UTF8String.fromString(new SimpleDateFormat(f.toString, Locale.US). - format(new java.util.Date(time.asInstanceOf[Long] * 1000L))) - ).getOrElse(null) + try { + UTF8String.fromString(DateTimeUtils.newDateFormat(f.toString, timeZone) + .format(new java.util.Date(time.asInstanceOf[Long] * 1000L))) + } catch { + case NonFatal(_) => null + } } } } } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val sdf = classOf[SimpleDateFormat].getName + val df = classOf[DateFormat].getName if (format.foldable) { if (formatter == null) { ExprCode("", "true", "(UTF8String) null") } else { - val formatterName = ctx.addReferenceObj("formatter", formatter, sdf) + val formatterName = ctx.addReferenceObj("formatter", formatter, df) val t = left.genCode(ctx) ev.copy(code = s""" ${t.code} @@ -662,14 +765,16 @@ case class FromUnixTime(sec: Expression, format: Expression) }""") } } else { + val tz = ctx.addReferenceMinorObj(timeZone) + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (seconds, f) => { s""" try { - ${ev.value} = UTF8String.fromString((new $sdf($f.toString())).format( + ${ev.value} = UTF8String.fromString($dtu.newDateFormat($f.toString(), $tz).format( new java.util.Date($seconds * 1000L))); } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; - }""".stripMargin + }""" }) } } @@ -776,8 +881,10 @@ case class NextDay(startDate: Expression, dayOfWeek: Expression) /** * Adds an interval to timestamp. */ -case class TimeAdd(start: Expression, interval: Expression) - extends BinaryExpression with ImplicitCastInputTypes { +case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[String] = None) + extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + + def this(start: Expression, interval: Expression) = this(start, interval, None) override def left: Expression = start override def right: Expression = interval @@ -788,16 +895,20 @@ case class TimeAdd(start: Expression, interval: Expression) override def dataType: DataType = TimestampType + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], itvl.months, itvl.microseconds) + start.asInstanceOf[Long], itvl.months, itvl.microseconds, timeZone) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val tz = ctx.addReferenceMinorObj(timeZone) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds)""" + s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $tz)""" }) } } @@ -863,8 +974,10 @@ case class FromUTCTimestamp(left: Expression, right: Expression) /** * Subtracts an interval from timestamp. */ -case class TimeSub(start: Expression, interval: Expression) - extends BinaryExpression with ImplicitCastInputTypes { +case class TimeSub(start: Expression, interval: Expression, timeZoneId: Option[String] = None) + extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + + def this(start: Expression, interval: Expression) = this(start, interval, None) override def left: Expression = start override def right: Expression = interval @@ -875,16 +988,20 @@ case class TimeSub(start: Expression, interval: Expression) override def dataType: DataType = TimestampType + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds) + start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, timeZone) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val tz = ctx.addReferenceMinorObj(timeZone) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds)""" + s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $tz)""" }) } } @@ -937,8 +1054,10 @@ case class AddMonths(startDate: Expression, numMonths: Expression) 3.94959677 """) // scalastyle:on line.size.limit -case class MonthsBetween(date1: Expression, date2: Expression) - extends BinaryExpression with ImplicitCastInputTypes { +case class MonthsBetween(date1: Expression, date2: Expression, timeZoneId: Option[String] = None) + extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + + def this(date1: Expression, date2: Expression) = this(date1, date2, None) override def left: Expression = date1 override def right: Expression = date2 @@ -947,14 +1066,18 @@ case class MonthsBetween(date1: Expression, date2: Expression) override def dataType: DataType = DoubleType + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + override def nullSafeEval(t1: Any, t2: Any): Any = { - DateTimeUtils.monthsBetween(t1.asInstanceOf[Long], t2.asInstanceOf[Long]) + DateTimeUtils.monthsBetween(t1.asInstanceOf[Long], t2.asInstanceOf[Long], timeZone) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val tz = ctx.addReferenceMinorObj(timeZone) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (l, r) => { - s"""$dtu.monthsBetween($l, $r)""" + s"""$dtu.monthsBetween($l, $r, $tz)""" }) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 6c38f4998e914..1b98c30d3760b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -204,6 +204,15 @@ case class Stack(children: Seq[Expression]) extends Generator { } } +case class GeneratorOuter(child: Generator) extends UnaryExpression with Generator { + final override def eval(input: InternalRow = null): TraversableOnce[InternalRow] = + throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + + final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + + override def elementSchema: StructType = child.elementSchema +} /** * A base class for [[Explode]] and [[PosExplode]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index 667ff649d1297..d55f85d56281c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -23,7 +23,6 @@ import scala.util.parsing.combinator.RegexParsers import com.fasterxml.jackson.core._ -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.InternalRow @@ -517,7 +516,10 @@ case class StructToJson(options: Map[String, String], child: Expression) @transient lazy val gen = - new JacksonGenerator(child.dataType.asInstanceOf[StructType], writer) + new JacksonGenerator( + child.dataType.asInstanceOf[StructType], + writer, + new JSONOptions(options)) override def dataType: DataType = StringType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index d24a502c9fbde..92036b727dbbd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 3fcbb05372d87..ac56ff13fa5bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCo import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils object InterpretedPredicate { @@ -86,6 +85,18 @@ trait PredicateHelper { */ protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = expr.references.subsetOf(plan.outputSet) + + /** + * Returns true iff `expr` could be evaluated as a condition within join. + */ + protected def canEvaluateWithinJoin(expr: Expression): Boolean = expr match { + case e: SubqueryExpression => + // non-correlated subquery will be replaced as literal + e.children.isEmpty + case a: AttributeReference => true + case e: Unevaluable => false + case e => e.children.forall(canEvaluateWithinJoin) + } } @ExpressionDescription( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala index 834897b85023d..26697e9867b35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala @@ -60,9 +60,11 @@ case class TableIdentifier(table: String, database: Option[String]) override val identifier: String = table def this(table: String) = this(table, None) - } +/** A fully qualified identifier for a table (i.e., database.tableName) */ +case class QualifiedTableName(database: String, name: String) + object TableIdentifier { def apply(tableName: String): TableIdentifier = new TableIdentifier(tableName) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 009c517ae4651..55d37cce99114 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -94,7 +94,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) CombineLimits, CombineUnions, // Constant folding and strength reduction - NullPropagation, + NullPropagation(conf), FoldablePropagation, OptimizeIn(conf), ConstantFolding, @@ -114,7 +114,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) Batch("Check Cartesian Products", Once, CheckCartesianProducts(conf)) :: Batch("Decimal Optimizations", fixedPoint, - DecimalAggregates) :: + DecimalAggregates(conf)) :: Batch("Typed Filter Optimization", fixedPoint, CombineTypedFilters) :: Batch("LocalRelation", fixedPoint, @@ -893,7 +893,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newRight = rightFilterConditions. reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val (newJoinConditions, others) = - commonFilterCondition.partition(e => !SubqueryExpression.hasCorrelatedSubquery(e)) + commonFilterCondition.partition(canEvaluateWithinJoin) val newJoinCond = (newJoinConditions ++ joinCondition).reduceLeftOption(And) val join = Join(newLeft, newRight, joinType, newJoinCond) @@ -1026,7 +1026,7 @@ case class CheckCartesianProducts(conf: CatalystConf) * This uses the same rules for increasing the precision and scale of the output as * [[org.apache.spark.sql.catalyst.analysis.DecimalPrecision]]. */ -object DecimalAggregates extends Rule[LogicalPlan] { +case class DecimalAggregates(conf: CatalystConf) extends Rule[LogicalPlan] { import Decimal.MAX_LONG_DIGITS /** Maximum number of decimal digits representable precisely in a Double */ @@ -1044,7 +1044,7 @@ object DecimalAggregates extends Rule[LogicalPlan] { we.copy(windowFunction = ae.copy(aggregateFunction = Average(UnscaledValue(e)))) Cast( Divide(newAggExpr, Literal.create(math.pow(10.0, scale), DoubleType)), - DecimalType(prec + 4, scale + 4)) + DecimalType(prec + 4, scale + 4), Option(conf.sessionLocalTimeZone)) case _ => we } @@ -1056,7 +1056,7 @@ object DecimalAggregates extends Rule[LogicalPlan] { val newAggExpr = ae.copy(aggregateFunction = Average(UnscaledValue(e))) Cast( Divide(newAggExpr, Literal.create(math.pow(10.0, scale), DoubleType)), - DecimalType(prec + 4, scale + 4)) + DecimalType(prec + 4, scale + 4), Option(conf.sessionLocalTimeZone)) case _ => ae } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 949ccdcb458cd..5bfc0ce9efcdd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -340,7 +340,7 @@ object LikeSimplification extends Rule[LogicalPlan] { * equivalent [[Literal]] values. This rule is more specific with * Null value propagation from bottom to top of the expression tree. */ -object NullPropagation extends Rule[LogicalPlan] { +case class NullPropagation(conf: CatalystConf) extends Rule[LogicalPlan] { private def nonNullLiteral(e: Expression): Boolean = e match { case Literal(null, _) => false case _ => true @@ -348,10 +348,10 @@ object NullPropagation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { - case e @ WindowExpression(Cast(Literal(0L, _), _), _) => - Cast(Literal(0L), e.dataType) + case e @ WindowExpression(Cast(Literal(0L, _), _, _), _) => + Cast(Literal(0L), e.dataType, Option(conf.sessionLocalTimeZone)) case e @ AggregateExpression(Count(exprs), _, _, _) if !exprs.exists(nonNullLiteral) => - Cast(Literal(0L), e.dataType) + Cast(Literal(0L), e.dataType, Option(conf.sessionLocalTimeZone)) case e @ IsNull(c) if !c.nullable => Literal.create(false, BooleanType) case e @ IsNotNull(c) if !c.nullable => Literal.create(true, BooleanType) case e @ GetArrayItem(Literal(null, _), _) => Literal.create(null, e.dataType) @@ -518,8 +518,8 @@ case class OptimizeCodegen(conf: CatalystConf) extends Rule[LogicalPlan] { */ object SimplifyCasts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case Cast(e, dataType) if e.dataType == dataType => e - case c @ Cast(e, dataType) => (e.dataType, dataType) match { + case Cast(e, dataType, _) if e.dataType == dataType => e + case c @ Cast(e, dataType, _) => (e.dataType, dataType) match { case (ArrayType(from, false), ArrayType(to, true)) if from == to => e case (MapType(fromKey, fromValue, false), MapType(toKey, toValue, true)) if fromKey == toKey && fromValue == toValue => e diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index f20eb958fe973..89e1dc9e322e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -17,10 +17,15 @@ package org.apache.spark.sql.catalyst.optimizer +import java.util.TimeZone + +import scala.collection.mutable + import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ @@ -41,13 +46,18 @@ object ReplaceExpressions extends Rule[LogicalPlan] { */ object ComputeCurrentTime extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { - val dateExpr = CurrentDate() + val currentDates = mutable.Map.empty[String, Literal] val timeExpr = CurrentTimestamp() - val currentDate = Literal.create(dateExpr.eval(EmptyRow), dateExpr.dataType) - val currentTime = Literal.create(timeExpr.eval(EmptyRow), timeExpr.dataType) + val timestamp = timeExpr.eval(EmptyRow).asInstanceOf[Long] + val currentTime = Literal.create(timestamp, timeExpr.dataType) plan transformAllExpressions { - case CurrentDate() => currentDate + case CurrentDate(Some(timeZoneId)) => + currentDates.getOrElseUpdate(timeZoneId, { + Literal.create( + DateTimeUtils.millisToDays(timestamp / 1000L, TimeZone.getTimeZone(timeZoneId)), + DateType) + }) case CurrentTimestamp() => currentTime } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 180ad2e0ad1fa..bfe529e21e9ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -46,8 +46,7 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { : LogicalPlan = { assert(input.size >= 2) if (input.size == 2) { - val (joinConditions, others) = conditions.partition( - e => !SubqueryExpression.hasCorrelatedSubquery(e)) + val (joinConditions, others) = conditions.partition(canEvaluateWithinJoin) val ((left, leftJoinType), (right, rightJoinType)) = (input(0), input(1)) val innerJoinType = (leftJoinType, rightJoinType) match { case (Inner, Inner) => Inner @@ -75,7 +74,7 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val joinedRefs = left.outputSet ++ right.outputSet val (joinConditions, others) = conditions.partition( - e => e.references.subsetOf(joinedRefs) && !SubqueryExpression.hasCorrelatedSubquery(e)) + e => e.references.subsetOf(joinedRefs) && canEvaluateWithinJoin(e)) val joined = Join(left, right, innerJoinType, joinConditions.reduceLeftOption(And)) // should not have reference to same logical plan diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index f14aaab72a98f..4d62cce9da0ac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -68,8 +68,14 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Note that will almost certainly be planned as a Broadcast Nested Loop join. // Use EXISTS if performance matters to you. val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - val anyNull = splitConjunctivePredicates(joinCond.get).map(IsNull).reduceLeft(Or) - Join(outerPlan, sub, LeftAnti, Option(Or(anyNull, joinCond.get))) + // Expand the NOT IN expression with the NULL-aware semantic + // to its full form. That is from: + // (a1,b1,...) = (a2,b2,...) + // to + // (a1=a2 OR isnull(a1=a2)) AND (b1=b2 OR isnull(b1=b2)) AND ... + val joinConds = splitConjunctivePredicates(joinCond.get) + val pairs = joinConds.map(c => Or(c, IsNull(c))).reduceLeft(And) + Join(outerPlan, sub, LeftAnti, Option(pairs)) case (p, predicate) => val (newCond, inputPlan) = rewriteExistentialExpr(Seq(predicate), p) Project(p.output, Filter(newCond.get, inputPlan)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index c5f92c59c88f4..0893af26738bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -17,15 +17,11 @@ package org.apache.spark.sql.catalyst.planning -import scala.annotation.tailrec -import scala.collection.mutable - import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.types.IntegerType /** * A pattern that matches any number of project or filter operations on top of another relational diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 853e9f3b076a2..818f4e5ed2ae5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.Attribute object JoinType { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala index 4224a7997c410..77309ce391a1a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.types.MetadataBuilder import org.apache.spark.unsafe.types.CalendarInterval diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 0587a592145be..93550e1fc32ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -344,7 +344,8 @@ abstract class UnaryNode extends LogicalPlan { sizeInBytes = 1 } - child.stats(conf).copy(sizeInBytes = sizeInBytes) + // Don't propagate rowCount and attributeStats, since they are not estimated here. + Statistics(sizeInBytes = sizeInBytes, isBroadcastable = child.stats(conf).isBroadcastable) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 48f68a6415bd2..432097d6218d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTypes} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.{AggregateEstimation, ProjectEstimation} +import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.{AggregateEstimation, EstimationUtils, ProjectEstimation} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -101,10 +101,17 @@ case class Generate( override def producedAttributes: AttributeSet = AttributeSet(generatorOutput) - def qualifiedGeneratorOutput: Seq[Attribute] = qualifier.map { q => - // prepend the new qualifier to the existed one - generatorOutput.map(a => a.withQualifier(Some(q))) - }.getOrElse(generatorOutput) + def qualifiedGeneratorOutput: Seq[Attribute] = { + val qualifiedOutput = qualifier.map { q => + // prepend the new qualifier to the existed one + generatorOutput.map(a => a.withQualifier(Some(q))) + }.getOrElse(generatorOutput) + val nullableOutput = qualifiedOutput.map { + // if outer, make all attributes nullable, otherwise keep existing nullability + a => a.withNullability(outer || a.nullable) + } + nullableOutput + } def output: Seq[Attribute] = { if (join) child.output ++ qualifiedGeneratorOutput else qualifiedGeneratorOutput @@ -534,7 +541,10 @@ case class Aggregate( override def computeStats(conf: CatalystConf): Statistics = { def simpleEstimation: Statistics = { if (groupingExpressions.isEmpty) { - super.computeStats(conf).copy(sizeInBytes = 1) + Statistics( + sizeInBytes = EstimationUtils.getOutputSize(output, outputRowCount = 1), + rowCount = Some(1), + isBroadcastable = child.stats(conf).isBroadcastable) } else { super.computeStats(conf) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 0ab4c9016623e..7ae50a0007fd1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -313,6 +313,42 @@ case class MapGroups( outputObjAttr: Attribute, child: LogicalPlan) extends UnaryNode with ObjectProducer +/** Internal class representing State */ +trait LogicalState[S] + +/** Factory for constructing new `MapGroups` nodes. */ +object MapGroupsWithState { + def apply[K: Encoder, V: Encoder, S: Encoder, U: Encoder]( + func: (Any, Iterator[Any], LogicalState[Any]) => Iterator[Any], + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + child: LogicalPlan): LogicalPlan = { + val mapped = new MapGroupsWithState( + func, + UnresolvedDeserializer(encoderFor[K].deserializer, groupingAttributes), + UnresolvedDeserializer(encoderFor[V].deserializer, dataAttributes), + groupingAttributes, + dataAttributes, + CatalystSerde.generateObjAttr[U], + encoderFor[S].resolveAndBind().deserializer, + encoderFor[S].namedExpressions, + child) + CatalystSerde.serialize[U](mapped) + } +} + +case class MapGroupsWithState( + func: (Any, Iterator[Any], LogicalState[Any]) => Iterator[Any], + keyDeserializer: Expression, + valueDeserializer: Expression, + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + outputObjAttr: Attribute, + stateDeserializer: Expression, + stateSerializer: Seq[NamedExpression], + child: LogicalPlan) extends UnaryNode with ObjectProducer + + /** Factory for constructing new `FlatMapGroupsInR` nodes. */ object FlatMapGroupsInR { def apply( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala index 21e94fc941a5c..ce74554c17010 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala @@ -53,7 +53,7 @@ object AggregateEstimation { val outputAttrStats = getOutputMap(childStats.attributeStats, agg.output) Some(Statistics( - sizeInBytes = getOutputSize(agg.output, outputAttrStats, outputRows), + sizeInBytes = getOutputSize(agg.output, outputRows, outputAttrStats), rowCount = Some(outputRows), attributeStats = outputAttrStats, isBroadcastable = childStats.isBroadcastable)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala index cf4452d0fdfba..e8b794212c10d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala @@ -37,8 +37,8 @@ object EstimationUtils { def getOutputSize( attributes: Seq[Attribute], - attrStats: AttributeMap[ColumnStat], - outputRowCount: BigInt): BigInt = { + outputRowCount: BigInt, + attrStats: AttributeMap[ColumnStat] = AttributeMap(Nil)): BigInt = { // We assign a generic overhead for a Row object, the actual overhead is different for different // Row format. val sizePerRow = 8 + attributes.map { attr => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala index 50b869ab3ae89..e9084ad8b859c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala @@ -36,7 +36,7 @@ object ProjectEstimation { val outputAttrStats = getOutputMap(AttributeMap(inputAttrStats.toSeq ++ aliasStats), project.output) Some(childStats.copy( - sizeInBytes = getOutputSize(project.output, outputAttrStats, childStats.rowCount.get), + sizeInBytes = getOutputSize(project.output, childStats.rowCount.get, outputAttrStats), attributeStats = outputAttrStats)) } else { None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 8cc16d662b603..8fec9dd9b48bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -68,7 +68,6 @@ object CurrentOrigin { def withOrigin[A](o: Origin)(f: => A): A = { set(o) val ret = try f finally { reset() } - reset() ret } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index a96a3b7af2912..af70efbb0a91b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -60,7 +60,7 @@ object DateTimeUtils { final val TimeZoneGMT = TimeZone.getTimeZone("GMT") final val MonthOf31Days = Set(1, 3, 5, 7, 8, 10, 12) - @transient lazy val defaultTimeZone = TimeZone.getDefault + def defaultTimeZone(): TimeZone = TimeZone.getDefault() // Reuse the Calendar object in each thread as it is expensive to create in each method call. private val threadLocalGmtCalendar = new ThreadLocal[Calendar] { @@ -69,20 +69,19 @@ object DateTimeUtils { } } - // Java TimeZone has no mention of thread safety. Use thread local instance to be safe. - private val threadLocalLocalTimeZone = new ThreadLocal[TimeZone] { - override protected def initialValue: TimeZone = { - Calendar.getInstance.getTimeZone - } - } - // `SimpleDateFormat` is not thread-safe. - val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { + private val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { override def initialValue(): SimpleDateFormat = { new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) } } + def getThreadLocalTimestampFormat(timeZone: TimeZone): DateFormat = { + val sdf = threadLocalTimestampFormat.get() + sdf.setTimeZone(timeZone) + sdf + } + // `SimpleDateFormat` is not thread-safe. private val threadLocalDateFormat = new ThreadLocal[DateFormat] { override def initialValue(): SimpleDateFormat = { @@ -90,28 +89,54 @@ object DateTimeUtils { } } + def getThreadLocalDateFormat(): DateFormat = { + val sdf = threadLocalDateFormat.get() + sdf.setTimeZone(defaultTimeZone()) + sdf + } + + def newDateFormat(formatString: String, timeZone: TimeZone): DateFormat = { + val sdf = new SimpleDateFormat(formatString, Locale.US) + sdf.setTimeZone(timeZone) + sdf + } + // we should use the exact day as Int, for example, (year, month, day) -> day def millisToDays(millisUtc: Long): SQLDate = { + millisToDays(millisUtc, defaultTimeZone()) + } + + def millisToDays(millisUtc: Long, timeZone: TimeZone): SQLDate = { // SPARK-6785: use Math.floor so negative number of days (dates before 1970) // will correctly work as input for function toJavaDate(Int) - val millisLocal = millisUtc + threadLocalLocalTimeZone.get().getOffset(millisUtc) + val millisLocal = millisUtc + timeZone.getOffset(millisUtc) Math.floor(millisLocal.toDouble / MILLIS_PER_DAY).toInt } // reverse of millisToDays def daysToMillis(days: SQLDate): Long = { + daysToMillis(days, defaultTimeZone()) + } + + def daysToMillis(days: SQLDate, timeZone: TimeZone): Long = { val millisLocal = days.toLong * MILLIS_PER_DAY - millisLocal - getOffsetFromLocalMillis(millisLocal, threadLocalLocalTimeZone.get()) + millisLocal - getOffsetFromLocalMillis(millisLocal, timeZone) } def dateToString(days: SQLDate): String = - threadLocalDateFormat.get.format(toJavaDate(days)) + getThreadLocalDateFormat.format(toJavaDate(days)) // Converts Timestamp to string according to Hive TimestampWritable convention. def timestampToString(us: SQLTimestamp): String = { + timestampToString(us, defaultTimeZone()) + } + + // Converts Timestamp to string according to Hive TimestampWritable convention. + def timestampToString(us: SQLTimestamp, timeZone: TimeZone): String = { val ts = toJavaTimestamp(us) val timestampString = ts.toString - val formatted = threadLocalTimestampFormat.get.format(ts) + val timestampFormat = getThreadLocalTimestampFormat(timeZone) + val formatted = timestampFormat.format(ts) if (timestampString.length > 19 && timestampString.substring(19) != ".0") { formatted + timestampString.substring(19) @@ -233,10 +258,14 @@ object DateTimeUtils { * `T[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us]+[h]h:[m]m` */ def stringToTimestamp(s: UTF8String): Option[SQLTimestamp] = { + stringToTimestamp(s, defaultTimeZone()) + } + + def stringToTimestamp(s: UTF8String, timeZone: TimeZone): Option[SQLTimestamp] = { if (s == null) { return None } - var timeZone: Option[Byte] = None + var tz: Option[Byte] = None val segments: Array[Int] = Array[Int](1, 1, 1, 0, 0, 0, 0, 0, 0) var i = 0 var currentSegmentValue = 0 @@ -289,12 +318,12 @@ object DateTimeUtils { segments(i) = currentSegmentValue currentSegmentValue = 0 i += 1 - timeZone = Some(43) + tz = Some(43) } else if (b == '-' || b == '+') { segments(i) = currentSegmentValue currentSegmentValue = 0 i += 1 - timeZone = Some(b) + tz = Some(b) } else if (b == '.' && i == 5) { segments(i) = currentSegmentValue currentSegmentValue = 0 @@ -349,11 +378,11 @@ object DateTimeUtils { return None } - val c = if (timeZone.isEmpty) { - Calendar.getInstance() + val c = if (tz.isEmpty) { + Calendar.getInstance(timeZone) } else { Calendar.getInstance( - TimeZone.getTimeZone(f"GMT${timeZone.get.toChar}${segments(7)}%02d:${segments(8)}%02d")) + TimeZone.getTimeZone(f"GMT${tz.get.toChar}${segments(7)}%02d:${segments(8)}%02d")) } c.set(Calendar.MILLISECOND, 0) @@ -452,7 +481,11 @@ object DateTimeUtils { } private def localTimestamp(microsec: SQLTimestamp): SQLTimestamp = { - absoluteMicroSecond(microsec) + defaultTimeZone.getOffset(microsec / 1000) * 1000L + localTimestamp(microsec, defaultTimeZone()) + } + + private def localTimestamp(microsec: SQLTimestamp, timeZone: TimeZone): SQLTimestamp = { + absoluteMicroSecond(microsec) + timeZone.getOffset(microsec / 1000) * 1000L } /** @@ -462,6 +495,13 @@ object DateTimeUtils { ((localTimestamp(microsec) / MICROS_PER_SECOND / 3600) % 24).toInt } + /** + * Returns the hour value of a given timestamp value. The timestamp is expressed in microseconds. + */ + def getHours(microsec: SQLTimestamp, timeZone: TimeZone): Int = { + ((localTimestamp(microsec, timeZone) / MICROS_PER_SECOND / 3600) % 24).toInt + } + /** * Returns the minute value of a given timestamp value. The timestamp is expressed in * microseconds. @@ -470,6 +510,14 @@ object DateTimeUtils { ((localTimestamp(microsec) / MICROS_PER_SECOND / 60) % 60).toInt } + /** + * Returns the minute value of a given timestamp value. The timestamp is expressed in + * microseconds. + */ + def getMinutes(microsec: SQLTimestamp, timeZone: TimeZone): Int = { + ((localTimestamp(microsec, timeZone) / MICROS_PER_SECOND / 60) % 60).toInt + } + /** * Returns the second value of a given timestamp value. The timestamp is expressed in * microseconds. @@ -478,6 +526,14 @@ object DateTimeUtils { ((localTimestamp(microsec) / MICROS_PER_SECOND) % 60).toInt } + /** + * Returns the second value of a given timestamp value. The timestamp is expressed in + * microseconds. + */ + def getSeconds(microsec: SQLTimestamp, timeZone: TimeZone): Int = { + ((localTimestamp(microsec, timeZone) / MICROS_PER_SECOND) % 60).toInt + } + private[this] def isLeapYear(year: Int): Boolean = { (year % 4) == 0 && ((year % 100) != 0 || (year % 400) == 0) } @@ -742,9 +798,23 @@ object DateTimeUtils { * Returns a timestamp value, expressed in microseconds since 1.1.1970 00:00:00. */ def timestampAddInterval(start: SQLTimestamp, months: Int, microseconds: Long): SQLTimestamp = { - val days = millisToDays(start / 1000L) + timestampAddInterval(start, months, microseconds, defaultTimeZone()) + } + + /** + * Add timestamp and full interval. + * Returns a timestamp value, expressed in microseconds since 1.1.1970 00:00:00. + */ + def timestampAddInterval( + start: SQLTimestamp, + months: Int, + microseconds: Long, + timeZone: TimeZone): SQLTimestamp = { + val days = millisToDays(start / 1000L, timeZone) val newDays = dateAddMonths(days, months) - daysToMillis(newDays) * 1000L + start - daysToMillis(days) * 1000L + microseconds + start + + daysToMillis(newDays, timeZone) * 1000L - daysToMillis(days, timeZone) * 1000L + + microseconds } /** @@ -758,10 +828,24 @@ object DateTimeUtils { * 8 digits. */ def monthsBetween(time1: SQLTimestamp, time2: SQLTimestamp): Double = { + monthsBetween(time1, time2, defaultTimeZone()) + } + + /** + * Returns number of months between time1 and time2. time1 and time2 are expressed in + * microseconds since 1.1.1970. + * + * If time1 and time2 having the same day of month, or both are the last day of month, + * it returns an integer (time under a day will be ignored). + * + * Otherwise, the difference is calculated based on 31 days per month, and rounding to + * 8 digits. + */ + def monthsBetween(time1: SQLTimestamp, time2: SQLTimestamp, timeZone: TimeZone): Double = { val millis1 = time1 / 1000L val millis2 = time2 / 1000L - val date1 = millisToDays(millis1) - val date2 = millisToDays(millis2) + val date1 = millisToDays(millis1, timeZone) + val date2 = millisToDays(millis2, timeZone) val (year1, monthInYear1, dayInMonth1, daysToMonthEnd1) = splitDate(date1) val (year2, monthInYear2, dayInMonth2, daysToMonthEnd2) = splitDate(date2) @@ -772,8 +856,8 @@ object DateTimeUtils { return (months1 - months2).toDouble } // milliseconds is enough for 8 digits precision on the right side - val timeInDay1 = millis1 - daysToMillis(date1) - val timeInDay2 = millis2 - daysToMillis(date2) + val timeInDay1 = millis1 - daysToMillis(date1, timeZone) + val timeInDay2 = millis2 - daysToMillis(date2, timeZone) val timesBetween = (timeInDay1 - timeInDay2).toDouble / MILLIS_PER_DAY val diff = (months1 - months2).toDouble + (dayInMonth1 - dayInMonth2 + timesBetween) / 31.0 // rounding to 8 digits @@ -896,7 +980,7 @@ object DateTimeUtils { */ def convertTz(ts: SQLTimestamp, fromZone: TimeZone, toZone: TimeZone): SQLTimestamp = { // We always use local timezone to parse or format a timestamp - val localZone = threadLocalLocalTimeZone.get() + val localZone = defaultTimeZone() val utcTs = if (fromZone.getID == localZone.getID) { ts } else { @@ -907,9 +991,9 @@ object DateTimeUtils { if (toZone.getID == localZone.getID) { utcTs } else { - val localTs2 = utcTs + toZone.getOffset(utcTs / 1000L) * 1000L // in toZone + val localTs = utcTs + toZone.getOffset(utcTs / 1000L) * 1000L // in toZone // treat it as local timezone, convert to UTC (we could get the expected human time back) - localTs2 - getOffsetFromLocalMillis(localTs2 / 1000L, localZone) * 1000L + localTs - getOffsetFromLocalMillis(localTs / 1000L, localZone) * 1000L } } @@ -934,7 +1018,6 @@ object DateTimeUtils { */ private[util] def resetThreadLocals(): Unit = { threadLocalGmtCalendar.remove() - threadLocalLocalTimeZone.remove() threadLocalTimestampFormat.remove() threadLocalDateFormat.remove() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index 76dbb7cf0aec1..da5775bc3753a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -129,6 +129,17 @@ protected[sql] abstract class AtomicType extends DataType { private[sql] val ordering: Ordering[InternalType] } +object AtomicType { + /** + * Enables matching against AtomicType for expressions: + * {{{ + * case Cast(child @ AtomicType(), StringType) => + * ... + * }}} + */ + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[AtomicType] +} + /** * Numeric data types. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 0205c13aa986d..ca0000a465c97 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -37,8 +37,9 @@ import org.apache.spark.util.Utils * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. * If a provided name does not have a matching field, it will be ignored. For the case - * of extracting a single StructField, a `null` will be returned. - * Example: + * of extracting a single [[StructField]], a `null` will be returned. + * + * Scala Example: * {{{ * import org.apache.spark.sql._ * import org.apache.spark.sql.types._ @@ -53,28 +54,30 @@ import org.apache.spark.util.Utils * val singleField = struct("b") * // singleField: StructField = StructField(b,LongType,false) * - * // This struct does not have a field called "d". null will be returned. - * val nonExisting = struct("d") - * // nonExisting: StructField = null + * // If this struct does not have a field called "d", it throws an exception. + * struct("d") + * // java.lang.IllegalArgumentException: Field "d" does not exist. + * // ... * * // Extract multiple StructFields. Field names are provided in a set. * // A StructType object will be returned. * val twoFields = struct(Set("b", "c")) * // twoFields: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * // StructType(StructField(b,LongType,false), StructField(c,BooleanType,false)) * - * // Any names without matching fields will be ignored. - * // For the case shown below, "d" will be ignored and - * // it is treated as struct(Set("b", "c")). - * val ignoreNonExisting = struct(Set("b", "c", "d")) - * // ignoreNonExisting: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * // Any names without matching fields will throw an exception. + * // For the case shown below, an exception is thrown due to "d". + * struct(Set("b", "c", "d")) + * // java.lang.IllegalArgumentException: Field "d" does not exist. + * // ... * }}} * - * A [[org.apache.spark.sql.Row]] object is used as a value of the StructType. - * Example: + * A [[org.apache.spark.sql.Row]] object is used as a value of the [[StructType]]. + * + * Scala Example: * {{{ * import org.apache.spark.sql._ + * import org.apache.spark.sql.types._ * * val innerStruct = * StructType( @@ -87,7 +90,6 @@ import org.apache.spark.util.Utils * * // Create a Row with the schema defined by struct * val row = Row(Row(1, 2, true)) - * // row: Row = [[1,2,true]] * }}} * * @since 1.3.0 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala index c33219c95b50a..5a944e763e099 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala @@ -78,8 +78,12 @@ abstract class UserDefinedType[UserType >: Null] extends DataType with Serializa */ override private[spark] def asNullable: UserDefinedType[UserType] = this - override private[sql] def acceptsType(dataType: DataType) = - this.getClass == dataType.getClass + override private[sql] def acceptsType(dataType: DataType) = dataType match { + case other: UserDefinedType[_] => + this.getClass == other.getClass || + this.userClass.isAssignableFrom(other.userClass) + case _ => false + } override def sql: String = sqlType.sql diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 817de48de2798..81a97dc1ff3f2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import java.util.TimeZone + import org.scalatest.ShouldMatchers import org.apache.spark.sql.catalyst.{SimpleCatalystConf, TableIdentifier} @@ -258,7 +260,8 @@ class AnalysisSuite extends AnalysisTest with ShouldMatchers { val c = testRelation2.output(2) val plan = testRelation2.select('c).orderBy(Floor('a).asc) - val expected = testRelation2.select(c, a).orderBy(Floor(a.cast(DoubleType)).asc).select(c) + val expected = testRelation2.select(c, a) + .orderBy(Floor(Cast(a, DoubleType, Option(TimeZone.getDefault().getID))).asc).select(c) checkAnalysis(plan, expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala index 2a0205bdc90fe..553b1598e7750 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import java.util.TimeZone + import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -32,7 +34,7 @@ class ResolveGroupingAnalyticsSuite extends AnalysisTest { lazy val unresolved_c = UnresolvedAttribute("c") lazy val gid = 'spark_grouping_id.int.withNullability(false) lazy val hive_gid = 'grouping__id.int.withNullability(false) - lazy val grouping_a = Cast(ShiftRight(gid, 1) & 1, ByteType) + lazy val grouping_a = Cast(ShiftRight(gid, 1) & 1, ByteType, Option(TimeZone.getDefault().getID)) lazy val nulInt = Literal(null, IntegerType) lazy val nulStr = Literal(null, StringType) lazy val r1 = LocalRelation(a, b, c) @@ -213,7 +215,8 @@ class ResolveGroupingAnalyticsSuite extends AnalysisTest { val originalPlan = Filter(Grouping(unresolved_a) === 0, GroupingSets(Seq(Seq(), Seq(unresolved_a), Seq(unresolved_a, unresolved_b)), Seq(unresolved_a, unresolved_b), r1, Seq(unresolved_a, unresolved_b))) - val expected = Project(Seq(a, b), Filter(Cast(grouping_a, IntegerType) === 0, + val expected = Project(Seq(a, b), + Filter(Cast(grouping_a, IntegerType, Option(TimeZone.getDefault().getID)) === 0, Aggregate(Seq(a, b, gid), Seq(a, b, gid), Expand( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index dbb1e3e70fcb5..110bd02eed08e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -916,6 +916,18 @@ class TypeCoercionSuite extends PlanTest { ruleTest(rules, Divide(1L, nullLit), Divide(Cast(1L, DoubleType), Cast(nullLit, DoubleType))) ruleTest(rules, Divide(nullLit, 1L), Divide(Cast(nullLit, DoubleType), Cast(1L, DoubleType))) } + + test("binary comparison with string promotion") { + ruleTest(PromoteStrings, + GreaterThan(Literal("123"), Literal(1)), + GreaterThan(Cast(Literal("123"), IntegerType), Literal(1))) + ruleTest(PromoteStrings, + LessThan(Literal(true), Literal("123")), + LessThan(Literal(true), Cast(Literal("123"), BooleanType))) + ruleTest(PromoteStrings, + EqualTo(Literal(Array(1, 2)), Literal("123")), + EqualTo(Literal(Array(1, 2)), Literal("123"))) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 58e69f9ebea05..dcdb1ae089328 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -199,12 +199,17 @@ class UnsupportedOperationsSuite extends SparkFunSuite { _.intersect(_), streamStreamSupported = false) - // Sort: supported only on batch subplans and on aggregation + complete output mode + // Sort: supported only on batch subplans and after aggregation on streaming plan + complete mode testUnaryOperatorInStreamingPlan("sort", Sort(Nil, true, _)) assertSupportedInStreamingPlan( - "sort - sort over aggregated data in Complete output mode", + "sort - sort after aggregation in Complete output mode", streamRelation.groupBy()(Count("*")).sortBy(), Complete) + assertNotSupportedInStreamingPlan( + "sort - sort before aggregation in Complete output mode", + streamRelation.sortBy().groupBy()(Count("*")), + Complete, + Seq("sort", "aggregat", "complete")) assertNotSupportedInStreamingPlan( "sort - sort over aggregated data in Update output mode", streamRelation.groupBy()(Count("*")).sortBy(), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 91f464b47aae2..a5d399a065589 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -156,8 +156,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac test("the table type of an external table should be EXTERNAL_TABLE") { val catalog = newBasicCatalog() - val table = - newTable("external_table1", "db2").copy(tableType = CatalogTableType.EXTERNAL) + val table = newTable("external_table1", "db2").copy(tableType = CatalogTableType.EXTERNAL) catalog.createTable(table, ignoreIfExists = false) val actual = catalog.getTable("db2", "external_table1") assert(actual.tableType === CatalogTableType.EXTERNAL) @@ -278,7 +277,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac schema = new StructType() .add("HelLo", "int", nullable = false) .add("WoRLd", "int", nullable = true), - provider = Some("hive"), + provider = Some(defaultProvider), partitionColumnNames = Seq("WoRLd"), bucketSpec = Some(BucketSpec(4, Seq("HelLo"), Nil))) catalog.createTable(tbl, ignoreIfExists = false) @@ -330,7 +329,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac .add("col2", "string") .add("partCol1", "int") .add("partCol2", "string"), - provider = Some("hive"), + provider = Some(defaultProvider), partitionColumnNames = Seq("partCol1", "partCol2")) catalog.createTable(table, ignoreIfExists = false) @@ -357,7 +356,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac .add("col2", "string") .add("partCol1", "int") .add("partCol2", "string"), - provider = Some("hive"), + provider = Some(defaultProvider), partitionColumnNames = Seq("partCol1", "partCol2")) catalog.createTable(table, ignoreIfExists = false) @@ -505,7 +504,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac .add("col2", "string") .add("partCol1", "int") .add("partCol2", "string"), - provider = Some("hive"), + provider = Some(defaultProvider), partitionColumnNames = Seq("partCol1", "partCol2")) catalog.createTable(table, ignoreIfExists = false) @@ -726,7 +725,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac tableType = CatalogTableType.MANAGED, storage = CatalogStorageFormat.empty, schema = new StructType().add("a", "int").add("b", "string"), - provider = Some("hive") + provider = Some(defaultProvider) ) catalog.createTable(table, ignoreIfExists = false) @@ -746,7 +745,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac Some(Utils.createTempDir().getAbsolutePath), None, None, None, false, Map.empty), schema = new StructType().add("a", "int").add("b", "string"), - provider = Some("hive") + provider = Some(defaultProvider) ) catalog.createTable(externalTable, ignoreIfExists = false) assert(!exists(db.locationUri, "external_table")) @@ -763,7 +762,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac .add("col2", "string") .add("partCol1", "int") .add("partCol2", "string"), - provider = Some("hive"), + provider = Some(defaultProvider), partitionColumnNames = Seq("partCol1", "partCol2")) catalog.createTable(table, ignoreIfExists = false) @@ -829,6 +828,7 @@ abstract class CatalogTestUtils { // Unimplemented methods val tableInputFormat: String val tableOutputFormat: String + val defaultProvider: String def newEmptyCatalog(): ExternalCatalog // These fields must be lazy because they rely on fields that are not implemented yet @@ -848,6 +848,8 @@ abstract class CatalogTestUtils { CatalogTablePartition(Map("a" -> "5", "b" -> "6", "c" -> "7"), storageFormat) lazy val partWithUnknownColumns = CatalogTablePartition(Map("a" -> "5", "unknown" -> "6"), storageFormat) + lazy val partWithEmptyValue = + CatalogTablePartition(Map("a" -> "3", "b" -> ""), storageFormat) lazy val funcClass = "org.apache.spark.myFunc" /** @@ -899,7 +901,7 @@ abstract class CatalogTestUtils { .add("col2", "string") .add("a", "int") .add("b", "string"), - provider = Some("hive"), + provider = Some(defaultProvider), partitionColumnNames = Seq("a", "b"), bucketSpec = Some(BucketSpec(4, Seq("col1"), Nil))) } @@ -917,7 +919,6 @@ abstract class CatalogTestUtils { .add("col2", "string") .add("a", "int") .add("b", "string"), - viewOriginalText = Some("SELECT * FROM tbl1"), viewText = Some("SELECT * FROM tbl1"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> viewDefaultDatabase)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalogSuite.scala index 0605daa3f9e98..eb3fc006b2b71 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalogSuite.scala @@ -24,6 +24,7 @@ class InMemoryCatalogSuite extends ExternalCatalogSuite { protected override val utils: CatalogTestUtils = new CatalogTestUtils { override val tableInputFormat: String = "org.apache.park.SequenceFileInputFormat" override val tableOutputFormat: String = "org.apache.park.SequenceFileOutputFormat" + override val defaultProvider: String = "parquet" override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index ae93dffbab8d2..db73f03c8bb73 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -37,6 +37,7 @@ class SessionCatalogSuite extends PlanTest { private val utils = new CatalogTestUtils { override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat" override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat" + override val defaultProvider: String = "parquet" override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog } @@ -436,7 +437,7 @@ class SessionCatalogSuite extends PlanTest { == SubqueryAlias("tbl1", SimpleCatalogRelation(metastoreTable1), None)) // Otherwise, we'll first look up a temporary table with the same name assert(sessionCatalog.lookupRelation(TableIdentifier("tbl1")) - == SubqueryAlias("tbl1", tempTable1, Some(TableIdentifier("tbl1")))) + == SubqueryAlias("tbl1", tempTable1, None)) // Then, if that does not exist, look up the relation in the current database sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false) assert(sessionCatalog.lookupRelation(TableIdentifier("tbl1")) @@ -462,7 +463,7 @@ class SessionCatalogSuite extends PlanTest { val tmpView = Range(1, 10, 2, 10) catalog.createTempView("vw1", tmpView, overrideIfExists = false) val plan = catalog.lookupRelation(TableIdentifier("vw1"), Option("range")) - assert(plan == SubqueryAlias("range", tmpView, Option(TableIdentifier("vw1")))) + assert(plan == SubqueryAlias("range", tmpView, None)) } test("look up view relation") { @@ -479,7 +480,7 @@ class SessionCatalogSuite extends PlanTest { // Look up a view using current database of the session catalog. sessionCatalog.setCurrentDatabase("db3") comparePlans(sessionCatalog.lookupRelation(TableIdentifier("view1")), - SubqueryAlias("view1", view, Some(TableIdentifier("view1")))) + SubqueryAlias("view1", view, Some(TableIdentifier("view1", Some("db3"))))) } test("table exists") { @@ -625,6 +626,13 @@ class SessionCatalogSuite extends PlanTest { } assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must match " + "the partition spec (a, b) defined in table '`db2`.`tbl2`'")) + e = intercept[AnalysisException] { + catalog.createPartitions( + TableIdentifier("tbl2", Some("db2")), + Seq(partWithEmptyValue, part1), ignoreIfExists = true) + } + assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " + + "empty partition column value")) } test("drop partitions") { @@ -722,6 +730,16 @@ class SessionCatalogSuite extends PlanTest { assert(e.getMessage.contains( "Partition spec is invalid. The spec (a, unknown) must be contained within " + "the partition spec (a, b) defined in table '`db2`.`tbl2`'")) + e = intercept[AnalysisException] { + catalog.dropPartitions( + TableIdentifier("tbl2", Some("db2")), + Seq(partWithEmptyValue.spec, part1.spec), + ignoreIfNotExists = false, + purge = false, + retainData = false) + } + assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " + + "empty partition column value")) } test("get partition") { @@ -767,6 +785,11 @@ class SessionCatalogSuite extends PlanTest { } assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must match " + "the partition spec (a, b) defined in table '`db2`.`tbl1`'")) + e = intercept[AnalysisException] { + catalog.getPartition(TableIdentifier("tbl1", Some("db2")), partWithEmptyValue.spec) + } + assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " + + "empty partition column value")) } test("rename partitions") { @@ -834,6 +857,13 @@ class SessionCatalogSuite extends PlanTest { } assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must match " + "the partition spec (a, b) defined in table '`db2`.`tbl1`'")) + e = intercept[AnalysisException] { + catalog.renamePartitions( + TableIdentifier("tbl1", Some("db2")), + Seq(part1.spec), Seq(partWithEmptyValue.spec)) + } + assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " + + "empty partition column value")) } test("alter partitions") { @@ -893,6 +923,11 @@ class SessionCatalogSuite extends PlanTest { } assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must match " + "the partition spec (a, b) defined in table '`db2`.`tbl1`'")) + e = intercept[AnalysisException] { + catalog.alterPartitions(TableIdentifier("tbl1", Some("db2")), Seq(partWithEmptyValue)) + } + assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " + + "empty partition column value")) } test("list partition names") { @@ -914,10 +949,24 @@ class SessionCatalogSuite extends PlanTest { test("list partition names with invalid partial partition spec") { val catalog = new SessionCatalog(newBasicCatalog()) - intercept[AnalysisException] { + var e = intercept[AnalysisException] { catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")), - Some(Map("unknown" -> "unknown"))) + Some(partWithMoreColumns.spec)) } + assert(e.getMessage.contains("Partition spec is invalid. The spec (a, b, c) must be " + + "contained within the partition spec (a, b) defined in table '`db2`.`tbl2`'")) + e = intercept[AnalysisException] { + catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")), + Some(partWithUnknownColumns.spec)) + } + assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must be " + + "contained within the partition spec (a, b) defined in table '`db2`.`tbl2`'")) + e = intercept[AnalysisException] { + catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")), + Some(partWithEmptyValue.spec)) + } + assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " + + "empty partition column value")) } test("list partitions") { @@ -937,10 +986,22 @@ class SessionCatalogSuite extends PlanTest { test("list partitions with invalid partial partition spec") { val catalog = new SessionCatalog(newBasicCatalog()) - intercept[AnalysisException] { - catalog.listPartitions( - TableIdentifier("tbl2", Some("db2")), Some(Map("unknown" -> "unknown"))) + var e = intercept[AnalysisException] { + catalog.listPartitions(TableIdentifier("tbl2", Some("db2")), Some(partWithMoreColumns.spec)) + } + assert(e.getMessage.contains("Partition spec is invalid. The spec (a, b, c) must be " + + "contained within the partition spec (a, b) defined in table '`db2`.`tbl2`'")) + e = intercept[AnalysisException] { + catalog.listPartitions(TableIdentifier("tbl2", Some("db2")), + Some(partWithUnknownColumns.spec)) + } + assert(e.getMessage.contains("Partition spec is invalid. The spec (a, unknown) must be " + + "contained within the partition spec (a, b) defined in table '`db2`.`tbl2`'")) + e = intercept[AnalysisException] { + catalog.listPartitions(TableIdentifier("tbl2", Some("db2")), Some(partWithEmptyValue.spec)) } + assert(e.getMessage.contains("Partition spec is invalid. The spec ([a=3, b=]) contains an " + + "empty partition column value")) } test("list partitions when database/table does not exist") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index b748595fc4f2d..8eccadbdd8afb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -18,12 +18,14 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} -import java.util.{Calendar, TimeZone} +import java.util.{Calendar, Locale, TimeZone} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -32,10 +34,10 @@ import org.apache.spark.unsafe.types.UTF8String */ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { - private def cast(v: Any, targetType: DataType): Cast = { + private def cast(v: Any, targetType: DataType, timeZoneId: Option[String] = None): Cast = { v match { - case lit: Expression => Cast(lit, targetType) - case _ => Cast(Literal(v), targetType) + case lit: Expression => Cast(lit, targetType, timeZoneId) + case _ => Cast(Literal(v), targetType, timeZoneId) } } @@ -45,7 +47,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } private def checkNullCast(from: DataType, to: DataType): Unit = { - checkEvaluation(Cast(Literal.create(null, from), to), null) + checkEvaluation(cast(Literal.create(null, from), to, Option("GMT")), null) } test("null cast") { @@ -107,108 +109,98 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("cast string to timestamp") { - checkEvaluation(Cast(Literal("123"), TimestampType), null) - - var c = Calendar.getInstance() - c.set(2015, 0, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - checkEvaluation(Cast(Literal("2015"), TimestampType), - new Timestamp(c.getTimeInMillis)) - c = Calendar.getInstance() - c.set(2015, 2, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - checkEvaluation(Cast(Literal("2015-03"), TimestampType), - new Timestamp(c.getTimeInMillis)) - c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - checkEvaluation(Cast(Literal("2015-03-18"), TimestampType), - new Timestamp(c.getTimeInMillis)) - checkEvaluation(Cast(Literal("2015-03-18 "), TimestampType), - new Timestamp(c.getTimeInMillis)) - checkEvaluation(Cast(Literal("2015-03-18T"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance() - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkEvaluation(Cast(Literal("2015-03-18 12:03:17"), TimestampType), - new Timestamp(c.getTimeInMillis)) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17Z"), TimestampType), - new Timestamp(c.getTimeInMillis)) - checkEvaluation(Cast(Literal("2015-03-18 12:03:17Z"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17-1:0"), TimestampType), - new Timestamp(c.getTimeInMillis)) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17-01:00"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17+07:30"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17+7:3"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance() - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkEvaluation(Cast(Literal("2015-03-18 12:03:17.123"), TimestampType), - new Timestamp(c.getTimeInMillis)) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17.123"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 456) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17.456Z"), TimestampType), - new Timestamp(c.getTimeInMillis)) - checkEvaluation(Cast(Literal("2015-03-18 12:03:17.456Z"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17.123-1:0"), TimestampType), - new Timestamp(c.getTimeInMillis)) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17.123-01:00"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17.123+07:30"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17.123+7:3"), TimestampType), - new Timestamp(c.getTimeInMillis)) - - checkEvaluation(Cast(Literal("2015-03-18 123142"), TimestampType), null) - checkEvaluation(Cast(Literal("2015-03-18T123123"), TimestampType), null) - checkEvaluation(Cast(Literal("2015-03-18X"), TimestampType), null) - checkEvaluation(Cast(Literal("2015/03/18"), TimestampType), null) - checkEvaluation(Cast(Literal("2015.03.18"), TimestampType), null) - checkEvaluation(Cast(Literal("20150318"), TimestampType), null) - checkEvaluation(Cast(Literal("2015-031-8"), TimestampType), null) - checkEvaluation(Cast(Literal("2015-03-18T12:03:17-0:70"), TimestampType), null) + for (tz <- ALL_TIMEZONES) { + def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { + checkEvaluation(cast(Literal(str), TimestampType, Option(tz.getID)), expected) + } + + checkCastStringToTimestamp("123", null) + + var c = Calendar.getInstance(tz) + c.set(2015, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 ", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18 12:03:17", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17-1:0", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17-01:00", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17+07:30", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17+7:3", new Timestamp(c.getTimeInMillis)) + + // tests for the string including milliseconds. + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp("2015-03-18 12:03:17.123", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17.123", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 456) + checkCastStringToTimestamp("2015-03-18T12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp("2015-03-18T12:03:17.123-1:0", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17.123-01:00", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp("2015-03-18T12:03:17.123+07:30", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp("2015-03-18T12:03:17.123+7:3", new Timestamp(c.getTimeInMillis)) + + checkCastStringToTimestamp("2015-03-18 123142", null) + checkCastStringToTimestamp("2015-03-18T123123", null) + checkCastStringToTimestamp("2015-03-18X", null) + checkCastStringToTimestamp("2015/03/18", null) + checkCastStringToTimestamp("2015.03.18", null) + checkCastStringToTimestamp("20150318", null) + checkCastStringToTimestamp("2015-031-8", null) + checkCastStringToTimestamp("2015-03-18T12:03:17-0:70", null) + } } test("cast from int") { @@ -316,30 +308,43 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { val zts = sd + " 00:00:00" val sts = sd + " 00:00:02" val nts = sts + ".1" - val ts = Timestamp.valueOf(nts) - - var c = Calendar.getInstance() - c.set(2015, 2, 8, 2, 30, 0) - checkEvaluation(cast(cast(new Timestamp(c.getTimeInMillis), StringType), TimestampType), - c.getTimeInMillis * 1000) - c = Calendar.getInstance() - c.set(2015, 10, 1, 2, 30, 0) - checkEvaluation(cast(cast(new Timestamp(c.getTimeInMillis), StringType), TimestampType), - c.getTimeInMillis * 1000) + val ts = withDefaultTimeZone(TimeZoneGMT)(Timestamp.valueOf(nts)) + + for (tz <- ALL_TIMEZONES) { + val timeZoneId = Option(tz.getID) + var c = Calendar.getInstance(TimeZoneGMT) + c.set(2015, 2, 8, 2, 30, 0) + checkEvaluation( + cast(cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, timeZoneId), + c.getTimeInMillis * 1000) + c = Calendar.getInstance(TimeZoneGMT) + c.set(2015, 10, 1, 2, 30, 0) + checkEvaluation( + cast(cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, timeZoneId), + c.getTimeInMillis * 1000) + } + + val gmtId = Option("GMT") checkEvaluation(cast("abdef", StringType), "abdef") checkEvaluation(cast("abdef", DecimalType.USER_DEFAULT), null) - checkEvaluation(cast("abdef", TimestampType), null) + checkEvaluation(cast("abdef", TimestampType, gmtId), null) checkEvaluation(cast("12.65", DecimalType.SYSTEM_DEFAULT), Decimal(12.65)) checkEvaluation(cast(cast(sd, DateType), StringType), sd) checkEvaluation(cast(cast(d, StringType), DateType), 0) - checkEvaluation(cast(cast(nts, TimestampType), StringType), nts) - checkEvaluation(cast(cast(ts, StringType), TimestampType), DateTimeUtils.fromJavaTimestamp(ts)) + checkEvaluation(cast(cast(nts, TimestampType, gmtId), StringType, gmtId), nts) + checkEvaluation( + cast(cast(ts, StringType, gmtId), TimestampType, gmtId), + DateTimeUtils.fromJavaTimestamp(ts)) // all convert to string type to check - checkEvaluation(cast(cast(cast(nts, TimestampType), DateType), StringType), sd) - checkEvaluation(cast(cast(cast(ts, DateType), TimestampType), StringType), zts) + checkEvaluation(cast(cast(cast(nts, TimestampType, gmtId), DateType, gmtId), StringType), sd) + checkEvaluation( + cast(cast(cast(ts, DateType, gmtId), TimestampType, gmtId), StringType, gmtId), + zts) checkEvaluation(cast(cast("abdef", BinaryType), StringType), "abdef") @@ -351,7 +356,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), 5.toShort) checkEvaluation( - cast(cast(cast(cast(cast(cast("5", TimestampType), ByteType), + cast(cast(cast(cast(cast(cast("5", TimestampType, gmtId), ByteType), DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), null) checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), @@ -466,7 +471,9 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(d, DecimalType.SYSTEM_DEFAULT), null) checkEvaluation(cast(d, DecimalType(10, 2)), null) checkEvaluation(cast(d, StringType), "1970-01-01") - checkEvaluation(cast(cast(d, TimestampType), StringType), "1970-01-01 00:00:00") + + val gmtId = Option("GMT") + checkEvaluation(cast(cast(d, TimestampType, gmtId), StringType, gmtId), "1970-01-01 00:00:00") } test("cast from timestamp") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 587022f0a2275..7ea0bec145481 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -25,8 +25,8 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.objects.{CreateExternalRow, GetExternalRowField, ValidateExternalType} -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils} +import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, CreateExternalRow, GetExternalRowField, ValidateExternalType} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.ThreadUtils @@ -313,4 +313,15 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-17160: field names are properly escaped by AssertTrue") { GenerateUnsafeProjection.generate(AssertTrue(Cast(Literal("\""), BooleanType)) :: Nil) } + + test("should not apply common subexpression elimination on conditional expressions") { + val row = InternalRow(null) + val bound = BoundReference(0, IntegerType, true) + val assertNotNull = AssertNotNull(bound, Nil) + val expr = If(IsNull(bound), Literal(1), Add(assertNotNull, assertNotNull)) + val projection = GenerateUnsafeProjection.generate( + Seq(expr), subexpressionEliminationEnabled = true) + // should not throw exception + projection(row) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 35cea25ba0b7d..9978f35a03810 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.util.{Calendar, Locale} +import java.util.{Calendar, Locale, TimeZone} import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -30,16 +32,29 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { import IntegralLiteralTestUtils._ + val TimeZonePST = TimeZone.getTimeZone("PST") + val TimeZoneJST = TimeZone.getTimeZone("JST") + + val gmtId = Option(TimeZoneGMT.getID) + val pstId = Option(TimeZonePST.getID) + val jstId = Option(TimeZoneJST.getID) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + sdf.setTimeZone(TimeZoneGMT) val sdfDate = new SimpleDateFormat("yyyy-MM-dd", Locale.US) + sdfDate.setTimeZone(TimeZoneGMT) val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) test("datetime function current_date") { - val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis()) - val cd = CurrentDate().eval(EmptyRow).asInstanceOf[Int] - val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis()) + val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT) + val cd = CurrentDate(gmtId).eval(EmptyRow).asInstanceOf[Int] + val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT) assert(d0 <= cd && cd <= d1 && d1 - d0 <= 1) + + val cdjst = CurrentDate(jstId).eval(EmptyRow).asInstanceOf[Int] + val cdpst = CurrentDate(pstId).eval(EmptyRow).asInstanceOf[Int] + assert(cdpst <= cd && cd <= cdjst) } test("datetime function current_timestamp") { @@ -50,9 +65,10 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("DayOfYear") { val sdfDay = new SimpleDateFormat("D", Locale.US) + + val c = Calendar.getInstance() (0 to 3).foreach { m => (0 to 5).foreach { i => - val c = Calendar.getInstance() c.set(2000, m, 28, 0, 0, 0) c.add(Calendar.DATE, i) checkEvaluation(DayOfYear(Literal(new Date(c.getTimeInMillis))), @@ -66,8 +82,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("Year") { checkEvaluation(Year(Literal.create(null, DateType)), null) checkEvaluation(Year(Literal(d)), 2015) - checkEvaluation(Year(Cast(Literal(sdfDate.format(d)), DateType)), 2015) - checkEvaluation(Year(Cast(Literal(ts), DateType)), 2013) + checkEvaluation(Year(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 2015) + checkEvaluation(Year(Cast(Literal(ts), DateType, gmtId)), 2013) val c = Calendar.getInstance() (2000 to 2002).foreach { y => @@ -86,8 +102,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("Quarter") { checkEvaluation(Quarter(Literal.create(null, DateType)), null) checkEvaluation(Quarter(Literal(d)), 2) - checkEvaluation(Quarter(Cast(Literal(sdfDate.format(d)), DateType)), 2) - checkEvaluation(Quarter(Cast(Literal(ts), DateType)), 4) + checkEvaluation(Quarter(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 2) + checkEvaluation(Quarter(Cast(Literal(ts), DateType, gmtId)), 4) val c = Calendar.getInstance() (2003 to 2004).foreach { y => @@ -106,13 +122,13 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("Month") { checkEvaluation(Month(Literal.create(null, DateType)), null) checkEvaluation(Month(Literal(d)), 4) - checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType)), 4) - checkEvaluation(Month(Cast(Literal(ts), DateType)), 11) + checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 4) + checkEvaluation(Month(Cast(Literal(ts), DateType, gmtId)), 11) + val c = Calendar.getInstance() (2003 to 2004).foreach { y => (0 to 3).foreach { m => (0 to 2 * 24).foreach { i => - val c = Calendar.getInstance() c.set(y, m, 28, 0, 0, 0) c.add(Calendar.HOUR_OF_DAY, i) checkEvaluation(Month(Literal(new Date(c.getTimeInMillis))), @@ -127,11 +143,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(DayOfMonth(Cast(Literal("2000-02-29"), DateType)), 29) checkEvaluation(DayOfMonth(Literal.create(null, DateType)), null) checkEvaluation(DayOfMonth(Literal(d)), 8) - checkEvaluation(DayOfMonth(Cast(Literal(sdfDate.format(d)), DateType)), 8) - checkEvaluation(DayOfMonth(Cast(Literal(ts), DateType)), 8) + checkEvaluation(DayOfMonth(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 8) + checkEvaluation(DayOfMonth(Cast(Literal(ts), DateType, gmtId)), 8) + val c = Calendar.getInstance() (1999 to 2000).foreach { y => - val c = Calendar.getInstance() c.set(y, 0, 1, 0, 0, 0) (0 to 365).foreach { d => c.add(Calendar.DATE, 1) @@ -143,72 +159,114 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Seconds") { - assert(Second(Literal.create(null, DateType)).resolved === false) - checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(Second(Literal(ts)), 15) + assert(Second(Literal.create(null, DateType), gmtId).resolved === false) + assert(Second(Cast(Literal(d), TimestampType), None).resolved === true) + checkEvaluation(Second(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) + checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType, gmtId), gmtId), 15) + checkEvaluation(Second(Literal(ts), gmtId), 15) val c = Calendar.getInstance() - (0 to 60 by 5).foreach { s => - c.set(2015, 18, 3, 3, 5, s) - checkEvaluation(Second(Literal(new Timestamp(c.getTimeInMillis))), - c.get(Calendar.SECOND)) + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + c.setTimeZone(tz) + (0 to 60 by 5).foreach { s => + c.set(2015, 18, 3, 3, 5, s) + checkEvaluation( + Second(Literal(new Timestamp(c.getTimeInMillis)), timeZoneId), + c.get(Calendar.SECOND)) + } + checkConsistencyBetweenInterpretedAndCodegen( + (child: Expression) => Second(child, timeZoneId), TimestampType) } - checkConsistencyBetweenInterpretedAndCodegen(Second, TimestampType) } test("WeekOfYear") { checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) checkEvaluation(WeekOfYear(Literal(d)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(sdfDate.format(d)), DateType)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType)), 45) - checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType)), 18) + checkEvaluation(WeekOfYear(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType, gmtId)), 45) + checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType, gmtId)), 18) checkConsistencyBetweenInterpretedAndCodegen(WeekOfYear, DateType) } test("DateFormat") { - checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal.create(null, StringType)), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal("y")), "2015") - checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") + checkEvaluation( + DateFormatClass(Literal.create(null, TimestampType), Literal("y"), gmtId), + null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId), + Literal.create(null, StringType), gmtId), null) + + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId), + Literal("y"), gmtId), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), gmtId), "2013") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId), + Literal("H"), gmtId), "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), gmtId), "13") + + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, pstId), + Literal("y"), pstId), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), pstId), "2013") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, pstId), + Literal("H"), pstId), "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), pstId), "5") + + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, jstId), + Literal("y"), jstId), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), jstId), "2013") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, jstId), + Literal("H"), jstId), "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), jstId), "22") } test("Hour") { - assert(Hour(Literal.create(null, DateType)).resolved === false) - checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) - checkEvaluation(Hour(Literal(ts)), 13) + assert(Hour(Literal.create(null, DateType), gmtId).resolved === false) + assert(Hour(Literal(ts), None).resolved === true) + checkEvaluation(Hour(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) + checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType, gmtId), gmtId), 13) + checkEvaluation(Hour(Literal(ts), gmtId), 13) val c = Calendar.getInstance() - (0 to 24).foreach { h => - (0 to 60 by 15).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, h, m, s) - checkEvaluation(Hour(Literal(new Timestamp(c.getTimeInMillis))), - c.get(Calendar.HOUR_OF_DAY)) + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + c.setTimeZone(tz) + (0 to 24).foreach { h => + (0 to 60 by 15).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, h, m, s) + checkEvaluation( + Hour(Literal(new Timestamp(c.getTimeInMillis)), timeZoneId), + c.get(Calendar.HOUR_OF_DAY)) + } } } + checkConsistencyBetweenInterpretedAndCodegen( + (child: Expression) => Hour(child, timeZoneId), TimestampType) } - checkConsistencyBetweenInterpretedAndCodegen(Hour, TimestampType) } test("Minute") { - assert(Minute(Literal.create(null, DateType)).resolved === false) - checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) - checkEvaluation(Minute(Literal(ts)), 10) + assert(Minute(Literal.create(null, DateType), gmtId).resolved === false) + assert(Minute(Literal(ts), None).resolved === true) + checkEvaluation(Minute(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) + checkEvaluation( + Minute(Cast(Literal(sdf.format(d)), TimestampType, gmtId), gmtId), 10) + checkEvaluation(Minute(Literal(ts), gmtId), 10) val c = Calendar.getInstance() - (0 to 60 by 5).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, 3, m, s) - checkEvaluation(Minute(Literal(new Timestamp(c.getTimeInMillis))), - c.get(Calendar.MINUTE)) + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + c.setTimeZone(tz) + (0 to 60 by 5).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, 3, m, s) + checkEvaluation( + Minute(Literal(new Timestamp(c.getTimeInMillis)), timeZoneId), + c.get(Calendar.MINUTE)) + } } + checkConsistencyBetweenInterpretedAndCodegen( + (child: Expression) => Minute(child, timeZoneId), TimestampType) } - checkConsistencyBetweenInterpretedAndCodegen(Minute, TimestampType) } test("date_add") { @@ -250,46 +308,86 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("time_add") { - checkEvaluation( - TimeAdd(Literal(Timestamp.valueOf("2016-01-29 10:00:00")), - Literal(new CalendarInterval(1, 123000L))), - DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2016-02-29 10:00:00.123"))) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.US) + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + sdf.setTimeZone(tz) - checkEvaluation( - TimeAdd(Literal.create(null, TimestampType), Literal(new CalendarInterval(1, 123000L))), - null) - checkEvaluation( - TimeAdd(Literal(Timestamp.valueOf("2016-01-29 10:00:00")), - Literal.create(null, CalendarIntervalType)), - null) - checkEvaluation( - TimeAdd(Literal.create(null, TimestampType), Literal.create(null, CalendarIntervalType)), - null) - checkConsistencyBetweenInterpretedAndCodegen(TimeAdd, TimestampType, CalendarIntervalType) + checkEvaluation( + TimeAdd( + Literal(new Timestamp(sdf.parse("2016-01-29 10:00:00.000").getTime)), + Literal(new CalendarInterval(1, 123000L)), + timeZoneId), + DateTimeUtils.fromJavaTimestamp( + new Timestamp(sdf.parse("2016-02-29 10:00:00.123").getTime))) + + checkEvaluation( + TimeAdd( + Literal.create(null, TimestampType), + Literal(new CalendarInterval(1, 123000L)), + timeZoneId), + null) + checkEvaluation( + TimeAdd( + Literal(new Timestamp(sdf.parse("2016-01-29 10:00:00.000").getTime)), + Literal.create(null, CalendarIntervalType), + timeZoneId), + null) + checkEvaluation( + TimeAdd( + Literal.create(null, TimestampType), + Literal.create(null, CalendarIntervalType), + timeZoneId), + null) + checkConsistencyBetweenInterpretedAndCodegen( + (start: Expression, interval: Expression) => TimeAdd(start, interval, timeZoneId), + TimestampType, CalendarIntervalType) + } } test("time_sub") { - checkEvaluation( - TimeSub(Literal(Timestamp.valueOf("2016-03-31 10:00:00")), - Literal(new CalendarInterval(1, 0))), - DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2016-02-29 10:00:00"))) - checkEvaluation( - TimeSub( - Literal(Timestamp.valueOf("2016-03-30 00:00:01")), - Literal(new CalendarInterval(1, 2000000.toLong))), - DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2016-02-28 23:59:59"))) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.US) + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + sdf.setTimeZone(tz) - checkEvaluation( - TimeSub(Literal.create(null, TimestampType), Literal(new CalendarInterval(1, 123000L))), - null) - checkEvaluation( - TimeSub(Literal(Timestamp.valueOf("2016-01-29 10:00:00")), - Literal.create(null, CalendarIntervalType)), - null) - checkEvaluation( - TimeSub(Literal.create(null, TimestampType), Literal.create(null, CalendarIntervalType)), - null) - checkConsistencyBetweenInterpretedAndCodegen(TimeSub, TimestampType, CalendarIntervalType) + checkEvaluation( + TimeSub( + Literal(new Timestamp(sdf.parse("2016-03-31 10:00:00.000").getTime)), + Literal(new CalendarInterval(1, 0)), + timeZoneId), + DateTimeUtils.fromJavaTimestamp( + new Timestamp(sdf.parse("2016-02-29 10:00:00.000").getTime))) + checkEvaluation( + TimeSub( + Literal(new Timestamp(sdf.parse("2016-03-30 00:00:01.000").getTime)), + Literal(new CalendarInterval(1, 2000000.toLong)), + timeZoneId), + DateTimeUtils.fromJavaTimestamp( + new Timestamp(sdf.parse("2016-02-28 23:59:59.000").getTime))) + + checkEvaluation( + TimeSub( + Literal.create(null, TimestampType), + Literal(new CalendarInterval(1, 123000L)), + timeZoneId), + null) + checkEvaluation( + TimeSub( + Literal(new Timestamp(sdf.parse("2016-01-29 10:00:00.000").getTime)), + Literal.create(null, CalendarIntervalType), + timeZoneId), + null) + checkEvaluation( + TimeSub( + Literal.create(null, TimestampType), + Literal.create(null, CalendarIntervalType), + timeZoneId), + null) + checkConsistencyBetweenInterpretedAndCodegen( + (start: Expression, interval: Expression) => TimeSub(start, interval, timeZoneId), + TimestampType, CalendarIntervalType) + } } test("add_months") { @@ -313,28 +411,44 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("months_between") { - checkEvaluation( - MonthsBetween(Literal(Timestamp.valueOf("1997-02-28 10:30:00")), - Literal(Timestamp.valueOf("1996-10-30 00:00:00"))), - 3.94959677) - checkEvaluation( - MonthsBetween(Literal(Timestamp.valueOf("2015-01-30 11:52:00")), - Literal(Timestamp.valueOf("2015-01-30 11:50:00"))), - 0.0) - checkEvaluation( - MonthsBetween(Literal(Timestamp.valueOf("2015-01-31 00:00:00")), - Literal(Timestamp.valueOf("2015-03-31 22:00:00"))), - -2.0) - checkEvaluation( - MonthsBetween(Literal(Timestamp.valueOf("2015-03-31 22:00:00")), - Literal(Timestamp.valueOf("2015-02-28 00:00:00"))), - 1.0) - val t = Literal(Timestamp.valueOf("2015-03-31 22:00:00")) - val tnull = Literal.create(null, TimestampType) - checkEvaluation(MonthsBetween(t, tnull), null) - checkEvaluation(MonthsBetween(tnull, t), null) - checkEvaluation(MonthsBetween(tnull, tnull), null) - checkConsistencyBetweenInterpretedAndCodegen(MonthsBetween, TimestampType, TimestampType) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + sdf.setTimeZone(tz) + + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("1997-02-28 10:30:00").getTime)), + Literal(new Timestamp(sdf.parse("1996-10-30 00:00:00").getTime)), + timeZoneId), + 3.94959677) + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("2015-01-30 11:52:00").getTime)), + Literal(new Timestamp(sdf.parse("2015-01-30 11:50:00").getTime)), + timeZoneId), + 0.0) + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("2015-01-31 00:00:00").getTime)), + Literal(new Timestamp(sdf.parse("2015-03-31 22:00:00").getTime)), + timeZoneId), + -2.0) + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("2015-03-31 22:00:00").getTime)), + Literal(new Timestamp(sdf.parse("2015-02-28 00:00:00").getTime)), + timeZoneId), + 1.0) + val t = Literal(Timestamp.valueOf("2015-03-31 22:00:00")) + val tnull = Literal.create(null, TimestampType) + checkEvaluation(MonthsBetween(t, tnull, timeZoneId), null) + checkEvaluation(MonthsBetween(tnull, t, timeZoneId), null) + checkEvaluation(MonthsBetween(tnull, tnull, timeZoneId), null) + checkConsistencyBetweenInterpretedAndCodegen( + (time1: Expression, time2: Expression) => MonthsBetween(time1, time2, timeZoneId), + TimestampType, TimestampType) + } } test("last_day") { @@ -398,7 +512,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { expected) } val date = Date.valueOf("2015-07-22") - Seq("yyyy", "YYYY", "year", "YEAR", "yy", "YY").foreach{ fmt => + Seq("yyyy", "YYYY", "year", "YEAR", "yy", "YY").foreach { fmt => testTrunc(date, fmt, Date.valueOf("2015-01-01")) } Seq("month", "MONTH", "mon", "MON", "mm", "MM").foreach { fmt => @@ -414,19 +528,32 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" val sdf2 = new SimpleDateFormat(fmt2, Locale.US) - checkEvaluation( - FromUnixTime(Literal(0L), Literal("yyyy-MM-dd HH:mm:ss")), sdf1.format(new Timestamp(0))) - checkEvaluation(FromUnixTime( - Literal(1000L), Literal("yyyy-MM-dd HH:mm:ss")), sdf1.format(new Timestamp(1000000))) - checkEvaluation( - FromUnixTime(Literal(-1000L), Literal(fmt2)), sdf2.format(new Timestamp(-1000000))) - checkEvaluation( - FromUnixTime(Literal.create(null, LongType), Literal.create(null, StringType)), null) - checkEvaluation( - FromUnixTime(Literal.create(null, LongType), Literal("yyyy-MM-dd HH:mm:ss")), null) - checkEvaluation(FromUnixTime(Literal(1000L), Literal.create(null, StringType)), null) - checkEvaluation( - FromUnixTime(Literal(0L), Literal("not a valid format")), null) + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + checkEvaluation( + FromUnixTime(Literal(0L), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + sdf1.format(new Timestamp(0))) + checkEvaluation(FromUnixTime( + Literal(1000L), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + sdf1.format(new Timestamp(1000000))) + checkEvaluation( + FromUnixTime(Literal(-1000L), Literal(fmt2), timeZoneId), + sdf2.format(new Timestamp(-1000000))) + checkEvaluation( + FromUnixTime(Literal.create(null, LongType), Literal.create(null, StringType), timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal.create(null, LongType), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal(1000L), Literal.create(null, StringType), timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal(0L), Literal("not a valid format"), timeZoneId), null) + } } test("unix_timestamp") { @@ -435,34 +562,53 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val sdf2 = new SimpleDateFormat(fmt2, Locale.US) val fmt3 = "yy-MM-dd" val sdf3 = new SimpleDateFormat(fmt3, Locale.US) - val date1 = Date.valueOf("2015-07-24") - checkEvaluation( - UnixTimestamp(Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss")), 0L) - checkEvaluation(UnixTimestamp( - Literal(sdf1.format(new Timestamp(1000000))), Literal("yyyy-MM-dd HH:mm:ss")), 1000L) - checkEvaluation( - UnixTimestamp(Literal(new Timestamp(1000000)), Literal("yyyy-MM-dd HH:mm:ss")), 1000L) - checkEvaluation( - UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss")), - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1)) / 1000L) - checkEvaluation( - UnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2)), -1000L) - checkEvaluation(UnixTimestamp( - Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3)), - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24"))) / 1000L) - val t1 = UnixTimestamp( - CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] - val t2 = UnixTimestamp( - CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] - assert(t2 - t1 <= 1) - checkEvaluation( - UnixTimestamp(Literal.create(null, DateType), Literal.create(null, StringType)), null) - checkEvaluation( - UnixTimestamp(Literal.create(null, DateType), Literal("yyyy-MM-dd HH:mm:ss")), null) - checkEvaluation(UnixTimestamp( - Literal(date1), Literal.create(null, StringType)), date1.getTime / 1000L) - checkEvaluation( - UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format")), null) + sdf3.setTimeZone(TimeZoneGMT) + + withDefaultTimeZone(TimeZoneGMT) { + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation(UnixTimestamp( + Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), 0L) + checkEvaluation(UnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp( + Literal(new Timestamp(1000000)), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz) / 1000L) + checkEvaluation( + UnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), + -1000L) + checkEvaluation(UnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), + DateTimeUtils.daysToMillis( + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz) / 1000L) + val t1 = UnixTimestamp( + CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] + val t2 = UnixTimestamp( + CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] + assert(t2 - t1 <= 1) + checkEvaluation( + UnixTimestamp( + Literal.create(null, DateType), Literal.create(null, StringType), timeZoneId), + null) + checkEvaluation( + UnixTimestamp(Literal.create(null, DateType), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + null) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz) / 1000L) + checkEvaluation( + UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) + } + } } test("to_unix_timestamp") { @@ -471,34 +617,51 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val sdf2 = new SimpleDateFormat(fmt2, Locale.US) val fmt3 = "yy-MM-dd" val sdf3 = new SimpleDateFormat(fmt3, Locale.US) - val date1 = Date.valueOf("2015-07-24") - checkEvaluation( - ToUnixTimestamp(Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss")), 0L) - checkEvaluation(ToUnixTimestamp( - Literal(sdf1.format(new Timestamp(1000000))), Literal("yyyy-MM-dd HH:mm:ss")), 1000L) - checkEvaluation( - ToUnixTimestamp(Literal(new Timestamp(1000000)), Literal("yyyy-MM-dd HH:mm:ss")), 1000L) - checkEvaluation( - ToUnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss")), - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1)) / 1000L) - checkEvaluation( - ToUnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2)), -1000L) - checkEvaluation(ToUnixTimestamp( - Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3)), - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24"))) / 1000L) - val t1 = ToUnixTimestamp( - CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] - val t2 = ToUnixTimestamp( - CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] - assert(t2 - t1 <= 1) - checkEvaluation( - ToUnixTimestamp(Literal.create(null, DateType), Literal.create(null, StringType)), null) - checkEvaluation( - ToUnixTimestamp(Literal.create(null, DateType), Literal("yyyy-MM-dd HH:mm:ss")), null) - checkEvaluation(ToUnixTimestamp( - Literal(date1), Literal.create(null, StringType)), date1.getTime / 1000L) - checkEvaluation( - ToUnixTimestamp(Literal("2015-07-24"), Literal("not a valid format")), null) + sdf3.setTimeZone(TimeZoneGMT) + + withDefaultTimeZone(TimeZoneGMT) { + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation(ToUnixTimestamp( + Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), 0L) + checkEvaluation(ToUnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + 1000L) + checkEvaluation(ToUnixTimestamp( + Literal(new Timestamp(1000000)), Literal("yyyy-MM-dd HH:mm:ss")), + 1000L) + checkEvaluation( + ToUnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz) / 1000L) + checkEvaluation( + ToUnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), + -1000L) + checkEvaluation(ToUnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), + DateTimeUtils.daysToMillis( + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz) / 1000L) + val t1 = ToUnixTimestamp( + CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] + val t2 = ToUnixTimestamp( + CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] + assert(t2 - t1 <= 1) + checkEvaluation(ToUnixTimestamp( + Literal.create(null, DateType), Literal.create(null, StringType), timeZoneId), null) + checkEvaluation( + ToUnixTimestamp( + Literal.create(null, DateType), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + null) + checkEvaluation(ToUnixTimestamp( + Literal(date1), Literal.create(null, StringType), timeZoneId), + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz) / 1000L) + checkEvaluation( + ToUnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) + } + } } test("datediff") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala index c587d4f632531..d617ad540d5ff 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala @@ -32,6 +32,38 @@ class ExpressionSetSuite extends SparkFunSuite { val aAndBSet = AttributeSet(aUpper :: bUpper :: Nil) + // An [AttributeReference] with almost the maximum hashcode, to make testing canonicalize rules + // like `case GreaterThan(l, r) if l.hashcode > r.hashcode => GreaterThan(r, l)` easier + val maxHash = + Canonicalize.ignoreNamesTypes( + AttributeReference("maxHash", IntegerType)(exprId = + new ExprId(4, NamedExpression.jvmId) { + // maxHash's hashcode is calculated based on this exprId's hashcode, so we set this + // exprId's hashCode to this specific value to make sure maxHash's hashcode is + // `Int.MaxValue` + override def hashCode: Int = -1030353449 + // We are implementing this equals() only because the style-checking rule "you should + // implement equals and hashCode together" requires us to + override def equals(obj: Any): Boolean = super.equals(obj) + })).asInstanceOf[AttributeReference] + assert(maxHash.hashCode() == Int.MaxValue) + + // An [AttributeReference] with almost the minimum hashcode, to make testing canonicalize rules + // like `case GreaterThan(l, r) if l.hashcode > r.hashcode => GreaterThan(r, l)` easier + val minHash = + Canonicalize.ignoreNamesTypes( + AttributeReference("minHash", IntegerType)(exprId = + new ExprId(5, NamedExpression.jvmId) { + // minHash's hashcode is calculated based on this exprId's hashcode, so we set this + // exprId's hashCode to this specific value to make sure minHash's hashcode is + // `Int.MinValue` + override def hashCode: Int = 1407330692 + // We are implementing this equals() only because the style-checking rule "you should + // implement equals and hashCode together" requires us to + override def equals(obj: Any): Boolean = super.equals(obj) + })).asInstanceOf[AttributeReference] + assert(minHash.hashCode() == Int.MinValue) + def setTest(size: Int, exprs: Expression*): Unit = { test(s"expect $size: ${exprs.mkString(", ")}") { val set = ExpressionSet(exprs) @@ -75,10 +107,14 @@ class ExpressionSetSuite extends SparkFunSuite { setTest(1, aUpper >= bUpper, bUpper <= aUpper) // `Not` canonicalization - setTest(1, Not(aUpper > 1), aUpper <= 1, Not(Literal(1) < aUpper), Literal(1) >= aUpper) - setTest(1, Not(aUpper < 1), aUpper >= 1, Not(Literal(1) > aUpper), Literal(1) <= aUpper) - setTest(1, Not(aUpper >= 1), aUpper < 1, Not(Literal(1) <= aUpper), Literal(1) > aUpper) - setTest(1, Not(aUpper <= 1), aUpper > 1, Not(Literal(1) >= aUpper), Literal(1) < aUpper) + setTest(1, Not(maxHash > 1), maxHash <= 1, Not(Literal(1) < maxHash), Literal(1) >= maxHash) + setTest(1, Not(minHash > 1), minHash <= 1, Not(Literal(1) < minHash), Literal(1) >= minHash) + setTest(1, Not(maxHash < 1), maxHash >= 1, Not(Literal(1) > maxHash), Literal(1) <= maxHash) + setTest(1, Not(minHash < 1), minHash >= 1, Not(Literal(1) > minHash), Literal(1) <= minHash) + setTest(1, Not(maxHash >= 1), maxHash < 1, Not(Literal(1) <= maxHash), Literal(1) > maxHash) + setTest(1, Not(minHash >= 1), minHash < 1, Not(Literal(1) <= minHash), Literal(1) > minHash) + setTest(1, Not(maxHash <= 1), maxHash > 1, Not(Literal(1) >= maxHash), Literal(1) < maxHash) + setTest(1, Not(minHash <= 1), minHash > 1, Not(Literal(1) >= minHash), Literal(1) < minHash) // Reordering AND/OR expressions setTest(1, aUpper > bUpper && aUpper <= 10, aUpper <= 10 && aUpper > bUpper) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala index 2db2a043e546a..c48730bd9d1cc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala @@ -97,9 +97,9 @@ class SubexpressionEliminationSuite extends SparkFunSuite { val add2 = Add(add, add) var equivalence = new EquivalentExpressions - equivalence.addExprTree(add, true) - equivalence.addExprTree(abs, true) - equivalence.addExprTree(add2, true) + equivalence.addExprTree(add) + equivalence.addExprTree(abs) + equivalence.addExprTree(add2) // Should only have one equivalence for `one + two` assert(equivalence.getAllEquivalentExprs.count(_.size > 1) == 1) @@ -115,10 +115,10 @@ class SubexpressionEliminationSuite extends SparkFunSuite { val mul2 = Multiply(mul, mul) val sqrt = Sqrt(mul2) val sum = Add(mul2, sqrt) - equivalence.addExprTree(mul, true) - equivalence.addExprTree(mul2, true) - equivalence.addExprTree(sqrt, true) - equivalence.addExprTree(sum, true) + equivalence.addExprTree(mul) + equivalence.addExprTree(mul2) + equivalence.addExprTree(sqrt) + equivalence.addExprTree(sum) // (one * two), (one * two) * (one * two) and sqrt( (one * two) * (one * two) ) should be found assert(equivalence.getAllEquivalentExprs.count(_.size > 1) == 3) @@ -126,30 +126,6 @@ class SubexpressionEliminationSuite extends SparkFunSuite { assert(equivalence.getEquivalentExprs(mul2).size == 3) assert(equivalence.getEquivalentExprs(sqrt).size == 2) assert(equivalence.getEquivalentExprs(sum).size == 1) - - // Some expressions inspired by TPCH-Q1 - // sum(l_quantity) as sum_qty, - // sum(l_extendedprice) as sum_base_price, - // sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, - // sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, - // avg(l_extendedprice) as avg_price, - // avg(l_discount) as avg_disc - equivalence = new EquivalentExpressions - val quantity = Literal(1) - val price = Literal(1.1) - val discount = Literal(.24) - val tax = Literal(0.1) - equivalence.addExprTree(quantity, false) - equivalence.addExprTree(price, false) - equivalence.addExprTree(Multiply(price, Subtract(Literal(1), discount)), false) - equivalence.addExprTree( - Multiply( - Multiply(price, Subtract(Literal(1), discount)), - Add(Literal(1), tax)), false) - equivalence.addExprTree(price, false) - equivalence.addExprTree(discount, false) - // quantity, price, discount and (price * (1 - discount)) - assert(equivalence.getAllEquivalentExprs.count(_.size > 1) == 4) } test("Expression equivalence - non deterministic") { @@ -167,11 +143,24 @@ class SubexpressionEliminationSuite extends SparkFunSuite { val add = Add(two, fallback) val equivalence = new EquivalentExpressions - equivalence.addExprTree(add, true) + equivalence.addExprTree(add) // the `two` inside `fallback` should not be added assert(equivalence.getAllEquivalentExprs.count(_.size > 1) == 0) assert(equivalence.getAllEquivalentExprs.count(_.size == 1) == 3) // add, two, explode } + + test("Children of conditional expressions") { + val condition = And(Literal(true), Literal(false)) + val add = Add(Literal(1), Literal(2)) + val ifExpr = If(condition, add, add) + + val equivalence = new EquivalentExpressions + equivalence.addExprTree(ifExpr) + // the `add` inside `If` should not be added + assert(equivalence.getAllEquivalentExprs.count(_.size > 1) == 0) + // only ifExpr and its predicate expression + assert(equivalence.getAllEquivalentExprs.count(_.size == 1) == 2) + } } case class CodegenFallbackExpression(child: Expression) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala index a313681eeb8f0..a0d489681fd9f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.sql.catalyst.SimpleCatalystConf import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -33,7 +34,7 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper Batch("AnalysisNodes", Once, EliminateSubqueryAliases) :: Batch("Constant Folding", FixedPoint(50), - NullPropagation, + NullPropagation(SimpleCatalystConf(caseSensitiveAnalysis = true)), ConstantFolding, BooleanSimplification, SimplifyBinaryComparison, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 8147d06969bbe..1b9db06014921 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -34,7 +34,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { Batch("AnalysisNodes", Once, EliminateSubqueryAliases) :: Batch("Constant Folding", FixedPoint(50), - NullPropagation, + NullPropagation(SimpleCatalystConf(caseSensitiveAnalysis = true)), ConstantFolding, BooleanSimplification, PruneFilters) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index 87ad81db11b64..276b8055b08d0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.sql.catalyst.SimpleCatalystConf import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -32,7 +33,7 @@ class CombiningLimitsSuite extends PlanTest { Batch("Combine Limit", FixedPoint(10), CombineLimits) :: Batch("Constant Folding", FixedPoint(10), - NullPropagation, + NullPropagation(SimpleCatalystConf(caseSensitiveAnalysis = true)), ConstantFolding, BooleanSimplification, SimplifyConditionals) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala index 711294ed61928..a491f4433370d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.sql.catalyst.SimpleCatalystConf import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ @@ -29,7 +30,7 @@ class DecimalAggregatesSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Decimal Optimizations", FixedPoint(100), - DecimalAggregates) :: Nil + DecimalAggregates(SimpleCatalystConf(caseSensitiveAnalysis = true))) :: Nil } val testRelation = LocalRelation('a.decimal(2, 1), 'b.decimal(12, 1)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index 0877207728b38..9daede1a5f957 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -34,7 +34,7 @@ class OptimizeInSuite extends PlanTest { Batch("AnalysisNodes", Once, EliminateSubqueryAliases) :: Batch("ConstantFolding", FixedPoint(10), - NullPropagation, + NullPropagation(SimpleCatalystConf(caseSensitiveAnalysis = true)), ConstantFolding, BooleanSimplification, OptimizeIn(SimpleCatalystConf(caseSensitiveAnalysis = true))) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala index a191aa8fee702..908b370408280 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.catalyst.plans +import java.util.TimeZone + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType} +import org.apache.spark.sql.types.{DataType, DoubleType, IntegerType, LongType, StringType} class ConstraintPropagationSuite extends SparkFunSuite { @@ -49,6 +51,10 @@ class ConstraintPropagationSuite extends SparkFunSuite { } } + private def castWithTimeZone(expr: Expression, dataType: DataType) = { + Cast(expr, dataType, Option(TimeZone.getDefault().getID)) + } + test("propagating constraints in filters") { val tr = LocalRelation('a.int, 'b.string, 'c.int) @@ -276,14 +282,15 @@ class ConstraintPropagationSuite extends SparkFunSuite { tr.where('a.attr === 'b.attr && 'c.attr + 100 > 'd.attr && IsNotNull(Cast(Cast(resolveColumn(tr, "e"), LongType), LongType))).analyze.constraints, - ExpressionSet(Seq(Cast(resolveColumn(tr, "a"), LongType) === resolveColumn(tr, "b"), - Cast(resolveColumn(tr, "c") + 100, LongType) > resolveColumn(tr, "d"), + ExpressionSet(Seq( + castWithTimeZone(resolveColumn(tr, "a"), LongType) === resolveColumn(tr, "b"), + castWithTimeZone(resolveColumn(tr, "c") + 100, LongType) > resolveColumn(tr, "d"), IsNotNull(resolveColumn(tr, "a")), IsNotNull(resolveColumn(tr, "b")), IsNotNull(resolveColumn(tr, "c")), IsNotNull(resolveColumn(tr, "d")), IsNotNull(resolveColumn(tr, "e")), - IsNotNull(Cast(Cast(resolveColumn(tr, "e"), LongType), LongType))))) + IsNotNull(castWithTimeZone(castWithTimeZone(resolveColumn(tr, "e"), LongType), LongType))))) } test("infer isnotnull constraints from compound expressions") { @@ -294,22 +301,25 @@ class ConstraintPropagationSuite extends SparkFunSuite { Cast( Cast(Cast(resolveColumn(tr, "e"), LongType), LongType), LongType))).analyze.constraints, ExpressionSet(Seq( - Cast(resolveColumn(tr, "a"), LongType) + resolveColumn(tr, "b") === - Cast(resolveColumn(tr, "c"), LongType), + castWithTimeZone(resolveColumn(tr, "a"), LongType) + resolveColumn(tr, "b") === + castWithTimeZone(resolveColumn(tr, "c"), LongType), IsNotNull(resolveColumn(tr, "a")), IsNotNull(resolveColumn(tr, "b")), IsNotNull(resolveColumn(tr, "c")), IsNotNull(resolveColumn(tr, "e")), - IsNotNull(Cast(Cast(Cast(resolveColumn(tr, "e"), LongType), LongType), LongType))))) + IsNotNull( + castWithTimeZone(castWithTimeZone(castWithTimeZone( + resolveColumn(tr, "e"), LongType), LongType), LongType))))) verifyConstraints( tr.where(('a.attr * 'b.attr + 100) === 'c.attr && 'd / 10 === 'e).analyze.constraints, ExpressionSet(Seq( - Cast(resolveColumn(tr, "a"), LongType) * resolveColumn(tr, "b") + Cast(100, LongType) === - Cast(resolveColumn(tr, "c"), LongType), - Cast(resolveColumn(tr, "d"), DoubleType) / - Cast(10, DoubleType) === - Cast(resolveColumn(tr, "e"), DoubleType), + castWithTimeZone(resolveColumn(tr, "a"), LongType) * resolveColumn(tr, "b") + + castWithTimeZone(100, LongType) === + castWithTimeZone(resolveColumn(tr, "c"), LongType), + castWithTimeZone(resolveColumn(tr, "d"), DoubleType) / + castWithTimeZone(10, DoubleType) === + castWithTimeZone(resolveColumn(tr, "e"), DoubleType), IsNotNull(resolveColumn(tr, "a")), IsNotNull(resolveColumn(tr, "b")), IsNotNull(resolveColumn(tr, "c")), @@ -319,11 +329,12 @@ class ConstraintPropagationSuite extends SparkFunSuite { verifyConstraints( tr.where(('a.attr * 'b.attr - 10) >= 'c.attr && 'd / 10 < 'e).analyze.constraints, ExpressionSet(Seq( - Cast(resolveColumn(tr, "a"), LongType) * resolveColumn(tr, "b") - Cast(10, LongType) >= - Cast(resolveColumn(tr, "c"), LongType), - Cast(resolveColumn(tr, "d"), DoubleType) / - Cast(10, DoubleType) < - Cast(resolveColumn(tr, "e"), DoubleType), + castWithTimeZone(resolveColumn(tr, "a"), LongType) * resolveColumn(tr, "b") - + castWithTimeZone(10, LongType) >= + castWithTimeZone(resolveColumn(tr, "c"), LongType), + castWithTimeZone(resolveColumn(tr, "d"), DoubleType) / + castWithTimeZone(10, DoubleType) < + castWithTimeZone(resolveColumn(tr, "e"), DoubleType), IsNotNull(resolveColumn(tr, "a")), IsNotNull(resolveColumn(tr, "b")), IsNotNull(resolveColumn(tr, "c")), @@ -333,9 +344,9 @@ class ConstraintPropagationSuite extends SparkFunSuite { verifyConstraints( tr.where('a.attr + 'b.attr - 'c.attr * 'd.attr > 'e.attr * 1000).analyze.constraints, ExpressionSet(Seq( - (Cast(resolveColumn(tr, "a"), LongType) + resolveColumn(tr, "b")) - - (Cast(resolveColumn(tr, "c"), LongType) * resolveColumn(tr, "d")) > - Cast(resolveColumn(tr, "e") * 1000, LongType), + (castWithTimeZone(resolveColumn(tr, "a"), LongType) + resolveColumn(tr, "b")) - + (castWithTimeZone(resolveColumn(tr, "c"), LongType) * resolveColumn(tr, "d")) > + castWithTimeZone(resolveColumn(tr, "e") * 1000, LongType), IsNotNull(resolveColumn(tr, "a")), IsNotNull(resolveColumn(tr, "b")), IsNotNull(resolveColumn(tr, "c")), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala index 41a4bc359e4ce..c0b9515ca7cd0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala @@ -90,6 +90,28 @@ class AggregateEstimationSuite extends StatsEstimationTestBase { expectedOutputRowCount = 0) } + test("non-cbo estimation") { + val attributes = Seq("key12").map(nameToAttr) + val child = StatsTestPlan( + outputList = attributes, + rowCount = 4, + // rowCount * (overhead + column size) + size = Some(4 * (8 + 4)), + attributeStats = AttributeMap(Seq("key12").map(nameToColInfo))) + + val noGroupAgg = Aggregate(groupingExpressions = Nil, + aggregateExpressions = Seq(Alias(Count(Literal(1)), "cnt")()), child) + assert(noGroupAgg.stats(conf.copy(cboEnabled = false)) == + // overhead + count result size + Statistics(sizeInBytes = 8 + 8, rowCount = Some(1))) + + val hasGroupAgg = Aggregate(groupingExpressions = attributes, + aggregateExpressions = attributes :+ Alias(Count(Literal(1)), "cnt")(), child) + assert(hasGroupAgg.stats(conf.copy(cboEnabled = false)) == + // From UnaryNode.computeStats, childSize * outputRowSize / childRowSize + Statistics(sizeInBytes = 48 * (8 + 4 + 8) / (8 + 4))) + } + private def checkAggStats( tableColumns: Seq[String], tableRowCount: BigInt, @@ -107,7 +129,7 @@ class AggregateEstimationSuite extends StatsEstimationTestBase { val expectedAttrStats = AttributeMap(groupByColumns.map(nameToColInfo)) val expectedStats = Statistics( - sizeInBytes = getOutputSize(testAgg.output, expectedAttrStats, expectedOutputRowCount), + sizeInBytes = getOutputSize(testAgg.output, expectedOutputRowCount, expectedAttrStats), rowCount = Some(expectedOutputRowCount), attributeStats = expectedAttrStats) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala index e6adb6700a656..a5fac4ba6f03c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala @@ -45,11 +45,12 @@ class StatsEstimationTestBase extends SparkFunSuite { protected case class StatsTestPlan( outputList: Seq[Attribute], rowCount: BigInt, - attributeStats: AttributeMap[ColumnStat]) extends LeafNode { + attributeStats: AttributeMap[ColumnStat], + size: Option[BigInt] = None) extends LeafNode { override def output: Seq[Attribute] = outputList override def computeStats(conf: CatalystConf): Statistics = Statistics( - // sizeInBytes in stats of StatsTestPlan is useless in cbo estimation, we just use a fake value - sizeInBytes = Int.MaxValue, + // If sizeInBytes is useless in testing, we just use a fake value + sizeInBytes = size.getOrElse(Int.MaxValue), rowCount = Some(rowCount), attributeStats = attributeStats) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index e0a9a0c3d5c00..9799817494f15 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -27,6 +27,8 @@ import org.apache.spark.unsafe.types.UTF8String class DateTimeUtilsSuite extends SparkFunSuite { + val TimeZonePST = TimeZone.getTimeZone("PST") + private[this] def getInUTCDays(timestamp: Long): Int = { val tz = TimeZone.getDefault ((timestamp + tz.getOffset(timestamp)) / MILLIS_PER_DAY).toInt @@ -177,180 +179,155 @@ class DateTimeUtilsSuite extends SparkFunSuite { } test("string to timestamp") { - var c = Calendar.getInstance() - c.set(1969, 11, 31, 16, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp(UTF8String.fromString("1969-12-31 16:00:00")).get === - c.getTimeInMillis * 1000) - c.set(1, 0, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp(UTF8String.fromString("0001")).get === - c.getTimeInMillis * 1000) - c = Calendar.getInstance() - c.set(2015, 2, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp(UTF8String.fromString("2015-03")).get === - c.getTimeInMillis * 1000) - c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18")).get === - c.getTimeInMillis * 1000) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18 ")).get === - c.getTimeInMillis * 1000) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18T")).get === - c.getTimeInMillis * 1000) - - c = Calendar.getInstance() - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18 12:03:17")).get === - c.getTimeInMillis * 1000) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18T12:03:17")).get === - c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT-13:53")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17-13:53")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18T12:03:17Z")).get === - c.getTimeInMillis * 1000) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18 12:03:17Z")).get === - c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18T12:03:17-1:0")).get === - c.getTimeInMillis * 1000) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17-01:00")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17+07:30")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17+07:03")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance() - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18 12:03:17.123")).get === c.getTimeInMillis * 1000) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17.123")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 456) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17.456Z")).get === c.getTimeInMillis * 1000) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18 12:03:17.456Z")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17.123-1:0")).get === c.getTimeInMillis * 1000) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17.123-01:00")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17.123+07:30")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17.123+07:30")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17.123121+7:30")).get === - c.getTimeInMillis * 1000 + 121) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17.12312+7:30")).get === - c.getTimeInMillis * 1000 + 120) - - c = Calendar.getInstance() - c.set(Calendar.HOUR_OF_DAY, 18) - c.set(Calendar.MINUTE, 12) - c.set(Calendar.SECOND, 15) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp( - UTF8String.fromString("18:12:15")).get === - c.getTimeInMillis * 1000) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(Calendar.HOUR_OF_DAY, 18) - c.set(Calendar.MINUTE, 12) - c.set(Calendar.SECOND, 15) - c.set(Calendar.MILLISECOND, 123) - assert(stringToTimestamp( - UTF8String.fromString("T18:12:15.12312+7:30")).get === - c.getTimeInMillis * 1000 + 120) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(Calendar.HOUR_OF_DAY, 18) - c.set(Calendar.MINUTE, 12) - c.set(Calendar.SECOND, 15) - c.set(Calendar.MILLISECOND, 123) - assert(stringToTimestamp( - UTF8String.fromString("18:12:15.12312+7:30")).get === - c.getTimeInMillis * 1000 + 120) - - c = Calendar.getInstance() - c.set(2011, 4, 6, 7, 8, 9) - c.set(Calendar.MILLISECOND, 100) - assert(stringToTimestamp( - UTF8String.fromString("2011-05-06 07:08:09.1000")).get === c.getTimeInMillis * 1000) - - assert(stringToTimestamp(UTF8String.fromString("238")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("00238")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18 123142")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18T123123")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-03-18X")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015/03/18")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015.03.18")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("20150318")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-031-8")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("02015-01-18")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("015-01-18")).isEmpty) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03.17-20:0")).isEmpty) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03.17-0:70")).isEmpty) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03.17-1:0:0")).isEmpty) + for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { + def checkStringToTimestamp(str: String, expected: Option[Long]): Unit = { + assert(stringToTimestamp(UTF8String.fromString(str), tz) === expected) + } - // Truncating the fractional seconds - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+00:00")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - assert(stringToTimestamp( - UTF8String.fromString("2015-03-18T12:03:17.123456789+0:00")).get === - c.getTimeInMillis * 1000 + 123456) + var c = Calendar.getInstance(tz) + c.set(1969, 11, 31, 16, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("1969-12-31 16:00:00", Option(c.getTimeInMillis * 1000)) + c.set(1, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("0001", Option(c.getTimeInMillis * 1000)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("2015-03", Option(c.getTimeInMillis * 1000)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("2015-03-18", Option(c.getTimeInMillis * 1000)) + checkStringToTimestamp("2015-03-18 ", Option(c.getTimeInMillis * 1000)) + checkStringToTimestamp("2015-03-18T", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("2015-03-18 12:03:17", Option(c.getTimeInMillis * 1000)) + checkStringToTimestamp("2015-03-18T12:03:17", Option(c.getTimeInMillis * 1000)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the tz parameter. + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-13:53")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("2015-03-18T12:03:17-13:53", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("2015-03-18T12:03:17Z", Option(c.getTimeInMillis * 1000)) + checkStringToTimestamp("2015-03-18 12:03:17Z", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("2015-03-18T12:03:17-1:0", Option(c.getTimeInMillis * 1000)) + checkStringToTimestamp("2015-03-18T12:03:17-01:00", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("2015-03-18T12:03:17+07:30", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("2015-03-18T12:03:17+07:03", Option(c.getTimeInMillis * 1000)) + + // tests for the string including milliseconds. + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkStringToTimestamp("2015-03-18 12:03:17.123", Option(c.getTimeInMillis * 1000)) + checkStringToTimestamp("2015-03-18T12:03:17.123", Option(c.getTimeInMillis * 1000)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the tz parameter. + c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 456) + checkStringToTimestamp("2015-03-18T12:03:17.456Z", Option(c.getTimeInMillis * 1000)) + checkStringToTimestamp("2015-03-18 12:03:17.456Z", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkStringToTimestamp("2015-03-18T12:03:17.123-1:0", Option(c.getTimeInMillis * 1000)) + checkStringToTimestamp("2015-03-18T12:03:17.123-01:00", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkStringToTimestamp( + "2015-03-18T12:03:17.123121+7:30", Option(c.getTimeInMillis * 1000 + 121)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkStringToTimestamp( + "2015-03-18T12:03:17.12312+7:30", Option(c.getTimeInMillis * 1000 + 120)) + + c = Calendar.getInstance(tz) + c.set(Calendar.HOUR_OF_DAY, 18) + c.set(Calendar.MINUTE, 12) + c.set(Calendar.SECOND, 15) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("18:12:15", Option(c.getTimeInMillis * 1000)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(Calendar.HOUR_OF_DAY, 18) + c.set(Calendar.MINUTE, 12) + c.set(Calendar.SECOND, 15) + c.set(Calendar.MILLISECOND, 123) + checkStringToTimestamp("T18:12:15.12312+7:30", Option(c.getTimeInMillis * 1000 + 120)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(Calendar.HOUR_OF_DAY, 18) + c.set(Calendar.MINUTE, 12) + c.set(Calendar.SECOND, 15) + c.set(Calendar.MILLISECOND, 123) + checkStringToTimestamp("18:12:15.12312+7:30", Option(c.getTimeInMillis * 1000 + 120)) + + c = Calendar.getInstance(tz) + c.set(2011, 4, 6, 7, 8, 9) + c.set(Calendar.MILLISECOND, 100) + checkStringToTimestamp("2011-05-06 07:08:09.1000", Option(c.getTimeInMillis * 1000)) + + checkStringToTimestamp("238", None) + checkStringToTimestamp("00238", None) + checkStringToTimestamp("2015-03-18 123142", None) + checkStringToTimestamp("2015-03-18T123123", None) + checkStringToTimestamp("2015-03-18X", None) + checkStringToTimestamp("2015/03/18", None) + checkStringToTimestamp("2015.03.18", None) + checkStringToTimestamp("20150318", None) + checkStringToTimestamp("2015-031-8", None) + checkStringToTimestamp("02015-01-18", None) + checkStringToTimestamp("015-01-18", None) + checkStringToTimestamp("2015-03-18T12:03.17-20:0", None) + checkStringToTimestamp("2015-03-18T12:03.17-0:70", None) + checkStringToTimestamp("2015-03-18T12:03.17-1:0:0", None) + + // Truncating the fractional seconds + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+00:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp( + "2015-03-18T12:03:17.123456789+0:00", Option(c.getTimeInMillis * 1000 + 123456)) + } } test("SPARK-15379: special invalid date string") { @@ -373,27 +350,35 @@ class DateTimeUtilsSuite extends SparkFunSuite { } test("hours") { - val c = Calendar.getInstance() + val c = Calendar.getInstance(TimeZonePST) c.set(2015, 2, 18, 13, 2, 11) - assert(getHours(c.getTimeInMillis * 1000) === 13) + assert(getHours(c.getTimeInMillis * 1000, TimeZonePST) === 13) + assert(getHours(c.getTimeInMillis * 1000, TimeZoneGMT) === 20) c.set(2015, 12, 8, 2, 7, 9) - assert(getHours(c.getTimeInMillis * 1000) === 2) + assert(getHours(c.getTimeInMillis * 1000, TimeZonePST) === 2) + assert(getHours(c.getTimeInMillis * 1000, TimeZoneGMT) === 10) } test("minutes") { - val c = Calendar.getInstance() + val c = Calendar.getInstance(TimeZonePST) c.set(2015, 2, 18, 13, 2, 11) - assert(getMinutes(c.getTimeInMillis * 1000) === 2) + assert(getMinutes(c.getTimeInMillis * 1000, TimeZonePST) === 2) + assert(getMinutes(c.getTimeInMillis * 1000, TimeZoneGMT) === 2) + assert(getMinutes(c.getTimeInMillis * 1000, TimeZone.getTimeZone("Australia/North")) === 32) c.set(2015, 2, 8, 2, 7, 9) - assert(getMinutes(c.getTimeInMillis * 1000) === 7) + assert(getMinutes(c.getTimeInMillis * 1000, TimeZonePST) === 7) + assert(getMinutes(c.getTimeInMillis * 1000, TimeZoneGMT) === 7) + assert(getMinutes(c.getTimeInMillis * 1000, TimeZone.getTimeZone("Australia/North")) === 37) } test("seconds") { - val c = Calendar.getInstance() + val c = Calendar.getInstance(TimeZonePST) c.set(2015, 2, 18, 13, 2, 11) - assert(getSeconds(c.getTimeInMillis * 1000) === 11) + assert(getSeconds(c.getTimeInMillis * 1000, TimeZonePST) === 11) + assert(getSeconds(c.getTimeInMillis * 1000, TimeZoneGMT) === 11) c.set(2015, 2, 8, 2, 7, 9) - assert(getSeconds(c.getTimeInMillis * 1000) === 9) + assert(getSeconds(c.getTimeInMillis * 1000, TimeZonePST) === 9) + assert(getSeconds(c.getTimeInMillis * 1000, TimeZoneGMT) === 9) } test("hours / minutes / seconds") { @@ -467,6 +452,21 @@ class DateTimeUtilsSuite extends SparkFunSuite { c2.set(Calendar.MILLISECOND, 123) val ts2 = c2.getTimeInMillis * 1000L assert(timestampAddInterval(ts1, 36, 123000) === ts2) + + val c3 = Calendar.getInstance(TimeZonePST) + c3.set(1997, 1, 27, 16, 0, 0) + c3.set(Calendar.MILLISECOND, 0) + val ts3 = c3.getTimeInMillis * 1000L + val c4 = Calendar.getInstance(TimeZonePST) + c4.set(2000, 1, 27, 16, 0, 0) + c4.set(Calendar.MILLISECOND, 123) + val ts4 = c4.getTimeInMillis * 1000L + val c5 = Calendar.getInstance(TimeZoneGMT) + c5.set(2000, 1, 29, 0, 0, 0) + c5.set(Calendar.MILLISECOND, 123) + val ts5 = c5.getTimeInMillis * 1000L + assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST) === ts4) + assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT) === ts5) } test("monthsBetween") { @@ -481,6 +481,17 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(monthsBetween(c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L) === -36) c2.set(1996, 2, 31, 0, 0, 0) assert(monthsBetween(c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L) === 11) + + val c3 = Calendar.getInstance(TimeZonePST) + c3.set(2000, 1, 28, 16, 0, 0) + val c4 = Calendar.getInstance(TimeZonePST) + c4.set(1997, 1, 28, 16, 0, 0) + assert( + monthsBetween(c3.getTimeInMillis * 1000L, c4.getTimeInMillis * 1000L, TimeZonePST) + === 36.0) + assert( + monthsBetween(c3.getTimeInMillis * 1000L, c4.getTimeInMillis * 1000L, TimeZoneGMT) + === 35.90322581) } test("from UTC timestamp") { @@ -537,6 +548,21 @@ class DateTimeUtilsSuite extends SparkFunSuite { } test("daysToMillis and millisToDays") { + val c = Calendar.getInstance(TimeZonePST) + + c.set(2015, 11, 31, 16, 0, 0) + assert(millisToDays(c.getTimeInMillis, TimeZonePST) === 16800) + assert(millisToDays(c.getTimeInMillis, TimeZoneGMT) === 16801) + + c.set(2015, 11, 31, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + assert(daysToMillis(16800, TimeZonePST) === c.getTimeInMillis) + + c.setTimeZone(TimeZoneGMT) + c.set(2015, 11, 31, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + assert(daysToMillis(16800, TimeZoneGMT) === c.getTimeInMillis) + // There are some days are skipped entirely in some timezone, skip them here. val skipped_days = Map[String, Int]( "Kwajalein" -> 8632, @@ -547,13 +573,11 @@ class DateTimeUtilsSuite extends SparkFunSuite { "Pacific/Kwajalein" -> 8632, "MIT" -> 15338) for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { - DateTimeTestUtils.withDefaultTimeZone(tz) { - val skipped = skipped_days.getOrElse(tz.getID, Int.MinValue) - (-20000 to 20000).foreach { d => - if (d != skipped) { - assert(millisToDays(daysToMillis(d)) === d, - s"Round trip of ${d} did not work in tz ${tz}") - } + val skipped = skipped_days.getOrElse(tz.getID, Int.MinValue) + (-20000 to 20000).foreach { d => + if (d != skipped) { + assert(millisToDays(daysToMillis(d, tz), tz) === d, + s"Round trip of ${d} did not work in tz ${tz}") } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index a3f581ff27790..38029552d13bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -166,10 +166,7 @@ class Column(val expr: Expression) extends Logging { // Leave an unaliased generator with an empty list of names since the analyzer will generate // the correct defaults after the nested expression's type has been resolved. - case explode: Explode => MultiAlias(explode, Nil) - case explode: PosExplode => MultiAlias(explode, Nil) - - case jt: JsonTuple => MultiAlias(jt, Nil) + case g: Generator => MultiAlias(g, Nil) case func: UnresolvedFunction => UnresolvedAlias(func, Some(Column.generateAlias)) @@ -177,7 +174,7 @@ class Column(val expr: Expression) extends Logging { // NamedExpression under this Cast. case c: Cast => c.transformUp { - case Cast(ne: NamedExpression, to) => UnresolvedAlias(Cast(ne, to)) + case c @ Cast(_: NamedExpression, _, _) => UnresolvedAlias(c) } match { case ne: NamedExpression => ne case other => Alias(expr, usePrettyExpression(expr).sql)() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 7fc03bd5ef372..ff1f0177e8ba0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -386,7 +386,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case relation: CatalogRelation if DDLUtils.isHiveTable(relation.catalogTable) => relation.catalogTable.identifier } - EliminateSubqueryAliases(catalog.lookupRelation(tableIdentWithDB)) match { + + val tableRelation = df.sparkSession.table(tableIdentWithDB).queryExecution.analyzed + EliminateSubqueryAliases(tableRelation) match { // check if the table is a data source table (the relation is a BaseRelation). case LogicalRelation(dest: BaseRelation, _, _) if srcRelations.contains(dest) => throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 1a7a5ba798077..391c34f1285ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql import java.io.CharArrayWriter +import java.sql.{Date, Timestamp} +import java.util.TimeZone import scala.collection.JavaConverters._ import scala.language.implicitConversions @@ -43,7 +45,7 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} -import org.apache.spark.sql.catalyst.util.usePrettyExpression +import org.apache.spark.sql.catalyst.util.{usePrettyExpression, DateTimeUtils} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand, GlobalTempView, LocalTempView} import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -250,6 +252,8 @@ class Dataset[T] private[sql]( val hasMoreData = takeResult.length > numRows val data = takeResult.take(numRows) + lazy val timeZone = TimeZone.getTimeZone(sparkSession.sessionState.conf.sessionLocalTimeZone) + // For array values, replace Seq and Array with square brackets // For cells that are beyond `truncate` characters, replace it with the // first `truncate-3` and "..." @@ -260,6 +264,10 @@ class Dataset[T] private[sql]( case binary: Array[Byte] => binary.map("%02X".format(_)).mkString("[", " ", "]") case array: Array[_] => array.mkString("[", ", ", "]") case seq: Seq[_] => seq.mkString("[", ", ", "]") + case d: Date => + DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d)) + case ts: Timestamp => + DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(ts), timeZone) case _ => cell.toString } if (truncate > 0 && str.length > truncate) { @@ -2003,10 +2011,7 @@ class Dataset[T] private[sql]( if (groupColExprIds.contains(attr.exprId)) { attr } else { - // Removing duplicate rows should not change output attributes. We should keep - // the original exprId of the attribute. Otherwise, to select a column in original - // dataset will cause analysis exception due to unresolved attribute. - Alias(new First(attr).toAggregateExpression(), attr.name)(exprId = attr.exprId) + Alias(new First(attr).toAggregateExpression(), attr.name)() } } Aggregate(groupCols, aggCols, logicalPlan) @@ -2591,8 +2596,8 @@ class Dataset[T] private[sql]( * * Global temporary view is cross-session. Its lifetime is the lifetime of the Spark application, * i.e. it will be automatically dropped when the application terminates. It's tied to a system - * preserved database `_global_temp`, and we must use the qualified name to refer a global temp - * view, e.g. `SELECT * FROM _global_temp.view1`. + * preserved database `global_temp`, and we must use the qualified name to refer a global temp + * view, e.g. `SELECT * FROM global_temp.view1`. * * @throws AnalysisException if the view name is invalid or already exists * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 395d709f26591..0b05f412a6d94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -218,6 +218,160 @@ class KeyValueGroupedDataset[K, V] private[sql]( mapGroups((key, data) => f.call(key, data.asJava))(encoder) } + /** + * ::Experimental:: + * (Scala-specific) + * Applies the given function to each group of data, while using an additional keyed state. + * For each unique group, the function will be passed the group key and an iterator that contains + * all of the elements in the group. The function can return an object of arbitrary type, and + * optionally update or remove the corresponding state. The returned object will form a new + * [[Dataset]]. + * + * This function can be applied on both batch and streaming Datasets. With a streaming dataset, + * this function will be once for each in every trigger. For each key, the updated state from the + * function call in a trigger will be the state available in the function call in the next + * trigger. However, for batch, `mapGroupsWithState` behaves exactly as `mapGroups` and the + * function is called only once per key without any prior state. + * + * There is no guaranteed ordering of values in the iterator in the function. + * + * This function does not support partial aggregation, and as a result requires shuffling all + * the data in the [[Dataset]]. + * + * Internally, the implementation will spill to disk if any given group is too large to fit into + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the memory + * constraints of their cluster. + * + * @see [[State]] for more details of how to update/remove state in the function. + * @since 2.1.1 + */ + @Experimental + @InterfaceStability.Evolving + def mapGroupsWithState[STATE: Encoder, OUT: Encoder]( + func: (K, Iterator[V], State[STATE]) => OUT): Dataset[OUT] = { + val f = (key: K, it: Iterator[V], s: State[STATE]) => Iterator(func(key, it, s)) + flatMapGroupsWithState[STATE, OUT](f) + } + + /** + * ::Experimental:: + * (Java-specific) + * Applies the given function to each group of data, while using an additional keyed state. + * For each unique group, the function will be passed the group key and an iterator that contains + * all of the elements in the group. The function can return an object of arbitrary type, and + * optionally update or remove the corresponding state. The returned object will form a new + * [[Dataset]]. + * + * This function can be applied on both batch and streaming Datasets. With a streaming dataset, + * this function will be once for each in every trigger. For each key, the updated state from the + * function call in a trigger will be the state available in the function call in the next + * trigger. However, for batch, `mapGroupsWithState` behaves exactly as `mapGroups` and the + * function is called only once per key without any prior state. + * + * There is no guaranteed ordering of values in the iterator in the function. + * + * This function does not support partial aggregation, and as a result requires shuffling all + * the data in the [[Dataset]]. + * + * Internally, the implementation will spill to disk if any given group is too large to fit into + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the memory + * constraints of their cluster. + * + * @see [[State]] for more details of how to update/remove state in the function. + * @since 2.1.1 + */ + @Experimental + @InterfaceStability.Evolving + def mapGroupsWithState[STATE, OUT]( + func: MapGroupsWithStateFunction[K, V, STATE, OUT], + stateEncoder: Encoder[STATE], + outputEncoder: Encoder[OUT]): Dataset[OUT] = { + val f = (key: K, it: Iterator[V], s: State[STATE]) => Iterator(func.call(key, it.asJava, s)) + flatMapGroupsWithState[STATE, OUT](f)(stateEncoder, outputEncoder) + } + + + /** + * ::Experimental:: + * (Scala-specific) + * Applies the given function to each group of data, while using an additional keyed state. + * For each unique group, the function will be passed the group key and an iterator that contains + * all of the elements in the group. The function can return an iteratior of object of arbitrary + * type, and optionally update or remove the corresponding state. The returned object will form a + * new [[Dataset]]. + * + * This function can be applied on both batch and streaming Datasets. With a streaming dataset, + * this function will be once for each in every trigger. For each key, the updated state from the + * function call in a trigger will be the state available in the function call in the next + * trigger. However, for batch, `mapGroupsWithState` behaves exactly as `mapGroups` and the + * function is called only once per key without any prior state. + * + * There is no guaranteed ordering of values in the iterator in the function. + * + * This function does not support partial aggregation, and as a result requires shuffling all + * the data in the [[Dataset]]. + * + * Internally, the implementation will spill to disk if any given group is too large to fit into + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the memory + * constraints of their cluster. + * + * @see [[State]] for more details of how to update/remove state in the function. + * @since 2.1.1 + */ + @Experimental + @InterfaceStability.Evolving + def flatMapGroupsWithState[STATE: Encoder, OUT: Encoder]( + func: (K, Iterator[V], State[STATE]) => Iterator[OUT]): Dataset[OUT] = { + Dataset[OUT]( + sparkSession, + MapGroupsWithState[K, V, STATE, OUT]( + func.asInstanceOf[(Any, Iterator[Any], LogicalState[Any]) => Iterator[Any]], + groupingAttributes, + dataAttributes, + logicalPlan)) + } + + /** + * ::Experimental:: + * (Java-specific) + * Applies the given function to each group of data, while using an additional keyed state. + * For each unique group, the function will be passed the group key and an iterator that contains + * all of the elements in the group. The function can return an iteratior of object of arbitrary + * type, and optionally update or remove the corresponding state. The returned object will form a + * new [[Dataset]]. + * + * This function can be applied on both batch and streaming Datasets. With a streaming dataset, + * this function will be once for each in every trigger. For each key, the updated state from the + * function call in a trigger will be the state available in the function call in the next + * trigger. However, for batch, `mapGroupsWithState` behaves exactly as `flatMapGroups` and the + * function is called only once per key without any prior state. + * + * There is no guaranteed ordering of values in the iterator in the function. + * + * This function does not support partial aggregation, and as a result requires shuffling all + * the data in the [[Dataset]]. + * + * Internally, the implementation will spill to disk if any given group is too large to fit into + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the memory + * constraints of their cluster. + * + * @see [[State]] for more details of how to update/remove state in the function. + * @since 2.1.1 + */ + @Experimental + @InterfaceStability.Evolving + def flatMapGroupsWithState[STATE, OUT]( + func: FlatMapGroupsWithStateFunction[K, V, STATE, OUT], + stateEncoder: Encoder[STATE], + outputEncoder: Encoder[OUT]): Dataset[OUT] = { + val f = (key: K, it: Iterator[V], s: State[STATE]) => func.call(key, it.asJava, s).asScala + flatMapGroupsWithState[STATE, OUT](f)(stateEncoder, outputEncoder) + } + /** * (Scala-specific) * Reduces the elements of each group of data using the specified binary function. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index 2caf723669f63..375df64d39734 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -259,7 +259,7 @@ abstract class SQLImplicits extends LowPrioritySQLImplicits { * Conflicting implicits are placed here to disambiguate resolution. * * Reasons for including specific implicits: - * newProductEncoder - to disambiguate for [[List]]s which are both [[Seq]] and [[Product]] + * newProductEncoder - to disambiguate for `List`s which are both `Seq` and `Product` */ trait LowPrioritySQLImplicits { /** @since 1.6.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/State.scala b/sql/core/src/main/scala/org/apache/spark/sql/State.scala new file mode 100644 index 0000000000000..65f621fa05e57 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/State.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.sql.catalyst.plans.logical.LogicalState + +/** + * :: Experimental :: + * + * Wrapper class for interacting with state data in `mapGroupsWithState` and + * `flatMapGroupsWithState` operations on + * [[org.apache.spark.sql.KeyValueGroupedDataset KeyValueGroupedDataset]]. + * + * @note Operations on state are not threadsafe. + * + * Scala example of using `State`: + * {{{ + * // A mapping function that maintains an integer state for string keys and returns a string. + * def mappingFunction(key: String, value: Iterable[Int], state: State[Int]): Option[String] = { + * // Check if state exists + * if (state.exists) { + * val existingState = state.get // Get the existing state + * val shouldRemove = ... // Decide whether to remove the state + * if (shouldRemove) { + * state.remove() // Remove the state + * } else { + * val newState = ... + * state.update(newState) // Set the new state + * } + * } else { + * val initialState = ... + * state.update(initialState) // Set the initial state + * } + * ... // return something + * } + * + * }}} + * + * Java example of using `State`: + * {{{ + * // A mapping function that maintains an integer state for string keys and returns a string. + * Function3, State, String> mappingFunction = + * new Function3, State, String>() { + * + * @Override + * public String call(String key, Optional value, State state) { + * if (state.exists()) { + * int existingState = state.get(); // Get the existing state + * boolean shouldRemove = ...; // Decide whether to remove the state + * if (shouldRemove) { + * state.remove(); // Remove the state + * } else { + * int newState = ...; + * state.update(newState); // Set the new state + * } + * } else { + * int initialState = ...; // Set the initial state + * state.update(initialState); + * } + * ... // return something + * } + * }; + * }}} + * + * @tparam S Type of the state + * @since 2.1.1 + */ +@Experimental +@InterfaceStability.Evolving +trait State[S] extends LogicalState[S] { + + def exists: Boolean + + def get(): S + + def update(newState: S): Unit + + def remove(): Unit + + @inline final def getOption(): Option[S] = if (exists) Some(get()) else None + + @inline final override def toString(): String = { + getOption.map { _.toString }.getOrElse("") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index d94185b390448..7abfa4ea37a74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -92,7 +92,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try($inputTypes).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) }""") @@ -109,9 +109,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends | * @since 1.3.0 | */ |def register(name: String, f: UDF$i[$extTypeArgs, _], returnType: DataType): Unit = { + | val func = f$anyCast.call($anyParams) | functionRegistry.registerFunction( | name, - | (e: Seq[Expression]) => ScalaUDF(f$anyCast.call($anyParams), returnType, e)) + | (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) |}""".stripMargin) } */ @@ -124,7 +125,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -137,7 +138,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -150,7 +151,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -163,7 +164,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -176,7 +177,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -189,7 +190,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -202,7 +203,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -215,7 +216,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -228,7 +229,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -241,7 +242,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -254,7 +255,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -267,7 +268,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -280,7 +281,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -293,7 +294,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -306,7 +307,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -319,7 +320,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -332,7 +333,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -345,7 +346,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -358,7 +359,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -371,7 +372,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: ScalaReflection.schemaFor[A19].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -384,7 +385,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: ScalaReflection.schemaFor[A19].dataType :: ScalaReflection.schemaFor[A20].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -397,7 +398,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: ScalaReflection.schemaFor[A19].dataType :: ScalaReflection.schemaFor[A20].dataType :: ScalaReflection.schemaFor[A21].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -410,7 +411,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = { val dataType = ScalaReflection.schemaFor[RT].dataType val inputTypes = Try(ScalaReflection.schemaFor[A1].dataType :: ScalaReflection.schemaFor[A2].dataType :: ScalaReflection.schemaFor[A3].dataType :: ScalaReflection.schemaFor[A4].dataType :: ScalaReflection.schemaFor[A5].dataType :: ScalaReflection.schemaFor[A6].dataType :: ScalaReflection.schemaFor[A7].dataType :: ScalaReflection.schemaFor[A8].dataType :: ScalaReflection.schemaFor[A9].dataType :: ScalaReflection.schemaFor[A10].dataType :: ScalaReflection.schemaFor[A11].dataType :: ScalaReflection.schemaFor[A12].dataType :: ScalaReflection.schemaFor[A13].dataType :: ScalaReflection.schemaFor[A14].dataType :: ScalaReflection.schemaFor[A15].dataType :: ScalaReflection.schemaFor[A16].dataType :: ScalaReflection.schemaFor[A17].dataType :: ScalaReflection.schemaFor[A18].dataType :: ScalaReflection.schemaFor[A19].dataType :: ScalaReflection.schemaFor[A20].dataType :: ScalaReflection.schemaFor[A21].dataType :: ScalaReflection.schemaFor[A22].dataType :: Nil).toOption - def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil)) + def builder(e: Seq[Expression]) = ScalaUDF(func, dataType, e, inputTypes.getOrElse(Nil), Some(name)) functionRegistry.registerFunction(name, builder) UserDefinedFunction(func, dataType, inputTypes) } @@ -488,9 +489,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -498,9 +500,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -508,9 +511,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -518,9 +522,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -528,9 +533,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -538,9 +544,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -548,9 +555,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -558,9 +566,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -568,9 +577,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -578,9 +588,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -588,9 +599,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -598,9 +610,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -608,9 +621,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -618,9 +632,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -628,9 +643,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -638,9 +654,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -648,9 +665,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -658,9 +676,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -668,9 +687,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -678,9 +698,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -688,9 +709,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } /** @@ -698,9 +720,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 1.3.0 */ def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { + val func = f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUDF(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + (e: Seq[Expression]) => ScalaUDF(func, returnType, e)) } // scalastyle:on line.size.limit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 6b061f8ab2740..50252db789d46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalog +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset} import org.apache.spark.sql.types.StructType @@ -187,82 +189,169 @@ abstract class Catalog { def functionExists(dbName: String, functionName: String): Boolean /** - * :: Experimental :: - * Creates an external table from the given path and returns the corresponding DataFrame. + * Creates a table from the given path and returns the corresponding DataFrame. * It will use the default data source configured by spark.sql.sources.default. * * @since 2.0.0 */ + @deprecated("use createTable instead.", "2.2.0") + def createExternalTable(tableName: String, path: String): DataFrame = { + createTable(tableName, path) + } + + /** + * :: Experimental :: + * Creates a table from the given path and returns the corresponding DataFrame. + * It will use the default data source configured by spark.sql.sources.default. + * + * @since 2.2.0 + */ @Experimental @InterfaceStability.Evolving - def createExternalTable(tableName: String, path: String): DataFrame + def createTable(tableName: String, path: String): DataFrame /** - * :: Experimental :: - * Creates an external table from the given path based on a data source - * and returns the corresponding DataFrame. + * Creates a table from the given path based on a data source and returns the corresponding + * DataFrame. * * @since 2.0.0 */ + @deprecated("use createTable instead.", "2.2.0") + def createExternalTable(tableName: String, path: String, source: String): DataFrame = { + createTable(tableName, path, source) + } + + /** + * :: Experimental :: + * Creates a table from the given path based on a data source and returns the corresponding + * DataFrame. + * + * @since 2.2.0 + */ @Experimental @InterfaceStability.Evolving - def createExternalTable(tableName: String, path: String, source: String): DataFrame + def createTable(tableName: String, path: String, source: String): DataFrame /** - * :: Experimental :: - * Creates an external table from the given path based on a data source and a set of options. + * Creates a table from the given path based on a data source and a set of options. * Then, returns the corresponding DataFrame. * * @since 2.0.0 */ + @deprecated("use createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + options: java.util.Map[String, String]): DataFrame = { + createTable(tableName, source, options) + } + + /** + * :: Experimental :: + * Creates a table from the given path based on a data source and a set of options. + * Then, returns the corresponding DataFrame. + * + * @since 2.2.0 + */ @Experimental @InterfaceStability.Evolving + def createTable( + tableName: String, + source: String, + options: java.util.Map[String, String]): DataFrame = { + createTable(tableName, source, options.asScala.toMap) + } + + /** + * (Scala-specific) + * Creates a table from the given path based on a data source and a set of options. + * Then, returns the corresponding DataFrame. + * + * @since 2.0.0 + */ + @deprecated("use createTable instead.", "2.2.0") def createExternalTable( tableName: String, source: String, - options: java.util.Map[String, String]): DataFrame + options: Map[String, String]): DataFrame = { + createTable(tableName, source, options) + } /** * :: Experimental :: * (Scala-specific) - * Creates an external table from the given path based on a data source and a set of options. + * Creates a table from the given path based on a data source and a set of options. * Then, returns the corresponding DataFrame. * - * @since 2.0.0 + * @since 2.2.0 */ @Experimental @InterfaceStability.Evolving - def createExternalTable( + def createTable( tableName: String, source: String, options: Map[String, String]): DataFrame /** * :: Experimental :: - * Create an external table from the given path based on a data source, a schema and - * a set of options. Then, returns the corresponding DataFrame. + * Create a table from the given path based on a data source, a schema and a set of options. + * Then, returns the corresponding DataFrame. * * @since 2.0.0 */ + @deprecated("use createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: java.util.Map[String, String]): DataFrame = { + createTable(tableName, source, schema, options) + } + + /** + * :: Experimental :: + * Create a table from the given path based on a data source, a schema and a set of options. + * Then, returns the corresponding DataFrame. + * + * @since 2.2.0 + */ @Experimental @InterfaceStability.Evolving + def createTable( + tableName: String, + source: String, + schema: StructType, + options: java.util.Map[String, String]): DataFrame = { + createTable(tableName, source, schema, options.asScala.toMap) + } + + /** + * (Scala-specific) + * Create a table from the given path based on a data source, a schema and a set of options. + * Then, returns the corresponding DataFrame. + * + * @since 2.0.0 + */ + @deprecated("use createTable instead.", "2.2.0") def createExternalTable( tableName: String, source: String, schema: StructType, - options: java.util.Map[String, String]): DataFrame + options: Map[String, String]): DataFrame = { + createTable(tableName, source, schema, options) + } /** * :: Experimental :: * (Scala-specific) - * Create an external table from the given path based on a data source, a schema and - * a set of options. Then, returns the corresponding DataFrame. + * Create a table from the given path based on a data source, a schema and a set of options. + * Then, returns the corresponding DataFrame. * - * @since 2.0.0 + * @since 2.2.0 */ @Experimental @InterfaceStability.Evolving - def createExternalTable( + def createTable( tableName: String, source: String, schema: StructType, @@ -291,8 +380,8 @@ abstract class Catalog { * * Global temporary view is cross-session. Its lifetime is the lifetime of the Spark application, * i.e. it will be automatically dropped when the application terminates. It's tied to a system - * preserved database `_global_temp`, and we must use the qualified name to refer a global temp - * view, e.g. `SELECT * FROM _global_temp.view1`. + * preserved database `global_temp`, and we must use the qualified name to refer a global temp + * view, e.g. `SELECT * FROM global_temp.view1`. * * @param viewName the name of the view to be dropped. * @return true if the view is dropped successfully, false otherwise. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index 0384c0f236028..d5a8566d078f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -369,7 +369,7 @@ class SQLBuilder private ( case ar: AttributeReference if groupByAttrMap.contains(ar) => groupByAttrMap(ar) case a @ Cast(BitwiseAnd( ShiftRight(ar: AttributeReference, Literal(value: Any, IntegerType)), - Literal(1, IntegerType)), ByteType) if ar == gid => + Literal(1, IntegerType)), ByteType, _) if ar == gid => // for converting an expression to its original SQL format grouping(col) val idx = groupByExprs.length - 1 - value.asInstanceOf[Int] groupByExprs.lift(idx).map(Grouping).getOrElse(a) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala new file mode 100644 index 0000000000000..04fba17be4bfa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.types.DataType + + +/** + * Helper trait for abstracting scan functionality using + * [[org.apache.spark.sql.execution.vectorized.ColumnarBatch]]es. + */ +private[sql] trait ColumnarBatchScan extends CodegenSupport { + + val inMemoryTableScan: InMemoryTableScanExec = null + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) + + /** + * Generate [[ColumnVector]] expressions for our parent to consume as rows. + * This is called once per [[ColumnarBatch]]. + */ + private def genCodeColumnVector( + ctx: CodegenContext, + columnVar: String, + ordinal: String, + dataType: DataType, + nullable: Boolean): ExprCode = { + val javaType = ctx.javaType(dataType) + val value = ctx.getValue(columnVar, dataType, ordinal) + val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } + val valueVar = ctx.freshName("value") + val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" + val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { + s""" + boolean $isNullVar = $columnVar.isNullAt($ordinal); + $javaType $valueVar = $isNullVar ? ${ctx.defaultValue(dataType)} : ($value); + """ + } else { + s"$javaType $valueVar = $value;" + }).trim + ExprCode(code, isNullVar, valueVar) + } + + /** + * Produce code to process the input iterator as [[ColumnarBatch]]es. + * This produces an [[UnsafeRow]] for each row in each batch. + */ + // TODO: return ColumnarBatch.Rows instead + override protected def doProduce(ctx: CodegenContext): String = { + val input = ctx.freshName("input") + // PhysicalRDD always just has one input + ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") + + // metrics + val numOutputRows = metricTerm(ctx, "numOutputRows") + val scanTimeMetric = metricTerm(ctx, "scanTime") + val scanTimeTotalNs = ctx.freshName("scanTime") + ctx.addMutableState("long", scanTimeTotalNs, s"$scanTimeTotalNs = 0;") + + val columnarBatchClz = "org.apache.spark.sql.execution.vectorized.ColumnarBatch" + val batch = ctx.freshName("batch") + ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;") + + val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" + val idx = ctx.freshName("batchIdx") + ctx.addMutableState("int", idx, s"$idx = 0;") + val colVars = output.indices.map(i => ctx.freshName("colInstance" + i)) + val columnAssigns = colVars.zipWithIndex.map { case (name, i) => + ctx.addMutableState(columnVectorClz, name, s"$name = null;") + s"$name = $batch.column($i);" + } + + val nextBatch = ctx.freshName("nextBatch") + ctx.addNewFunction(nextBatch, + s""" + |private void $nextBatch() throws java.io.IOException { + | long getBatchStart = System.nanoTime(); + | if ($input.hasNext()) { + | $batch = ($columnarBatchClz)$input.next(); + | $numOutputRows.add($batch.numRows()); + | $idx = 0; + | ${columnAssigns.mkString("", "\n", "\n")} + | } + | $scanTimeTotalNs += System.nanoTime() - getBatchStart; + |}""".stripMargin) + + ctx.currentVars = null + val rowidx = ctx.freshName("rowIdx") + val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => + genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) + } + s""" + |if ($batch == null) { + | $nextBatch(); + |} + |while ($batch != null) { + | int numRows = $batch.numRows(); + | while ($idx < numRows) { + | int $rowidx = $idx++; + | ${consume(ctx, columnsBatchInput).trim} + | if (shouldStop()) return; + | } + | $batch = null; + | $nextBatch(); + |} + |$scanTimeMetric.add($scanTimeTotalNs / (1000 * 1000)); + |$scanTimeTotalNs = 0; + """.stripMargin + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 76161643976aa..39b010efec7b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -145,7 +145,7 @@ case class FileSourceScanExec( partitionFilters: Seq[Expression], dataFilters: Seq[Filter], override val metastoreTableIdentifier: Option[TableIdentifier]) - extends DataSourceScanExec { + extends DataSourceScanExec with ColumnarBatchScan { val supportsBatch: Boolean = relation.fileFormat.supportBatch( relation.sparkSession, StructType.fromAttributes(output)) @@ -312,7 +312,7 @@ case class FileSourceScanExec( override protected def doProduce(ctx: CodegenContext): String = { if (supportsBatch) { - return doProduceVectorized(ctx) + return super.doProduce(ctx) } val numOutputRows = metricTerm(ctx, "numOutputRows") // PhysicalRDD always just has one input @@ -336,88 +336,6 @@ case class FileSourceScanExec( """.stripMargin } - // Support codegen so that we can avoid the UnsafeRow conversion in all cases. Codegen - // never requires UnsafeRow as input. - private def doProduceVectorized(ctx: CodegenContext): String = { - val input = ctx.freshName("input") - // PhysicalRDD always just has one input - ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") - - // metrics - val numOutputRows = metricTerm(ctx, "numOutputRows") - val scanTimeMetric = metricTerm(ctx, "scanTime") - val scanTimeTotalNs = ctx.freshName("scanTime") - ctx.addMutableState("long", scanTimeTotalNs, s"$scanTimeTotalNs = 0;") - - val columnarBatchClz = "org.apache.spark.sql.execution.vectorized.ColumnarBatch" - val batch = ctx.freshName("batch") - ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;") - - val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" - val idx = ctx.freshName("batchIdx") - ctx.addMutableState("int", idx, s"$idx = 0;") - val colVars = output.indices.map(i => ctx.freshName("colInstance" + i)) - val columnAssigns = colVars.zipWithIndex.map { case (name, i) => - ctx.addMutableState(columnVectorClz, name, s"$name = null;") - s"$name = $batch.column($i);" - } - - val nextBatch = ctx.freshName("nextBatch") - ctx.addNewFunction(nextBatch, - s""" - |private void $nextBatch() throws java.io.IOException { - | long getBatchStart = System.nanoTime(); - | if ($input.hasNext()) { - | $batch = ($columnarBatchClz)$input.next(); - | $numOutputRows.add($batch.numRows()); - | $idx = 0; - | ${columnAssigns.mkString("", "\n", "\n")} - | } - | $scanTimeTotalNs += System.nanoTime() - getBatchStart; - |}""".stripMargin) - - ctx.currentVars = null - val rowidx = ctx.freshName("rowIdx") - val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => - genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) - } - s""" - |if ($batch == null) { - | $nextBatch(); - |} - |while ($batch != null) { - | int numRows = $batch.numRows(); - | while ($idx < numRows) { - | int $rowidx = $idx++; - | ${consume(ctx, columnsBatchInput).trim} - | if (shouldStop()) return; - | } - | $batch = null; - | $nextBatch(); - |} - |$scanTimeMetric.add($scanTimeTotalNs / (1000 * 1000)); - |$scanTimeTotalNs = 0; - """.stripMargin - } - - private def genCodeColumnVector(ctx: CodegenContext, columnVar: String, ordinal: String, - dataType: DataType, nullable: Boolean): ExprCode = { - val javaType = ctx.javaType(dataType) - val value = ctx.getValue(columnVar, dataType, ordinal) - val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } - val valueVar = ctx.freshName("value") - val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" - val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { - s""" - boolean ${isNullVar} = ${columnVar}.isNullAt($ordinal); - $javaType ${valueVar} = ${isNullVar} ? ${ctx.defaultValue(dataType)} : ($value); - """ - } else { - s"$javaType ${valueVar} = $value;" - }).trim - ExprCode(code, isNullVar, valueVar) - } - /** * Create an RDD for bucketed reads. * The non-bucketed variant of this function is [[createNonBucketedReadRDD]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 04b16af4ea261..69be7094d2c39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -162,11 +162,15 @@ case class GenerateExec( val index = ctx.freshName("index") // Add a check if the generate outer flag is true. - val checks = optionalCode(outer, data.isNull) + val checks = optionalCode(outer, s"($index == -1)") // Add position val position = if (e.position) { - Seq(ExprCode("", "false", index)) + if (outer) { + Seq(ExprCode("", s"$index == -1", index)) + } else { + Seq(ExprCode("", "false", index)) + } } else { Seq.empty } @@ -177,7 +181,14 @@ case class GenerateExec( val row = codeGenAccessor(ctx, data.value, "col", index, st, nullable, checks) val fieldChecks = checks ++ optionalCode(nullable, row.isNull) val columns = st.fields.toSeq.zipWithIndex.map { case (f, i) => - codeGenAccessor(ctx, row.value, f.name, i.toString, f.dataType, f.nullable, fieldChecks) + codeGenAccessor( + ctx, + row.value, + s"st_col${i}", + i.toString, + f.dataType, + f.nullable, + fieldChecks) } ("", row.code, columns) @@ -243,7 +254,7 @@ case class GenerateExec( val values = e.dataType match { case ArrayType(st: StructType, nullable) => st.fields.toSeq.zipWithIndex.map { case (f, i) => - codeGenAccessor(ctx, current, f.name, s"$i", f.dataType, f.nullable, checks) + codeGenAccessor(ctx, current, s"st_col${i}", s"$i", f.dataType, f.nullable, checks) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala index 1b7fedca8484c..b8ac070e3a959 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.internal.SQLConf @@ -104,7 +105,9 @@ case class OptimizeMetadataOnlyQuery( val partAttrs = getPartitionAttrs(relation.catalogTable.partitionColumnNames, relation) val partitionData = catalog.listPartitions(relation.catalogTable.identifier).map { p => InternalRow.fromSeq(partAttrs.map { attr => - Cast(Literal(p.spec(attr.name)), attr.dataType).eval() + // TODO: use correct timezone for partition values. + Cast(Literal(p.spec(attr.name)), attr.dataType, + Option(DateTimeUtils.defaultTimeZone().getID)).eval() }) } LocalRelation(partAttrs, partitionData) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index b3ef29f6e34c4..9d046c0766aa5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.execution import java.nio.charset.StandardCharsets -import java.sql.Timestamp +import java.sql.{Date, Timestamp} +import java.util.TimeZone import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} @@ -108,21 +109,18 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { /** - * Returns the result as a hive compatible sequence of strings. For native commands, the - * execution is simply passed back to Hive. + * Returns the result as a hive compatible sequence of strings. This is for testing only. */ def hiveResultString(): Seq[String] = executedPlan match { case ExecutedCommandExec(desc: DescribeTableCommand) => - SQLExecution.withNewExecutionId(sparkSession, this) { - // If it is a describe command for a Hive table, we want to have the output format - // be similar with Hive. - desc.run(sparkSession).map { - case Row(name: String, dataType: String, comment) => - Seq(name, dataType, - Option(comment.asInstanceOf[String]).getOrElse("")) - .map(s => String.format(s"%-20s", s)) - .mkString("\t") - } + // If it is a describe command for a Hive table, we want to have the output format + // be similar with Hive. + desc.run(sparkSession).map { + case Row(name: String, dataType: String, comment) => + Seq(name, dataType, + Option(comment.asInstanceOf[String]).getOrElse("")) + .map(s => String.format(s"%-20s", s)) + .mkString("\t") } // SHOW TABLES in Hive only output table names, while ours outputs database, table name, isTemp. case command: ExecutedCommandExec if command.cmd.isInstanceOf[ShowTablesCommand] => @@ -130,13 +128,11 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { case command: ExecutedCommandExec => command.executeCollect().map(_.getString(0)) case other => - SQLExecution.withNewExecutionId(sparkSession, this) { - val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq - // We need the types so we can output struct field names - val types = analyzed.output.map(_.dataType) - // Reformat to match hive tab delimited output. - result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq - } + val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq + // We need the types so we can output struct field names + val types = analyzed.output.map(_.dataType) + // Reformat to match hive tab delimited output. + result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")) } /** Formats a datum (based on the given data type) and returns the string representation. */ @@ -144,22 +140,6 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) - /** Implementation following Hive's TimestampWritable.toString */ - def formatTimestamp(timestamp: Timestamp): String = { - val timestampString = timestamp.toString - if (timestampString.length() > 19) { - if (timestampString.length() == 21) { - if (timestampString.substring(19).compareTo(".0") == 0) { - return DateTimeUtils.threadLocalTimestampFormat.get().format(timestamp) - } - } - return DateTimeUtils.threadLocalTimestampFormat.get().format(timestamp) + - timestampString.substring(19) - } - - return DateTimeUtils.threadLocalTimestampFormat.get().format(timestamp) - } - def formatDecimal(d: java.math.BigDecimal): String = { if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { java.math.BigDecimal.ZERO.toPlainString @@ -200,8 +180,11 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" - case (d: Int, DateType) => new java.util.Date(DateTimeUtils.daysToMillis(d)).toString - case (t: Timestamp, TimestampType) => formatTimestamp(t) + case (d: Date, DateType) => + DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d)) + case (t: Timestamp, TimestampType) => + DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t), + TimeZone.getTimeZone(sparkSession.sessionState.conf.sessionLocalTimeZone)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) case (other, tpe) if primitiveTypes.contains(tpe) => other.toString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index fafb91967086f..b0bb1f6c00305 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, EventTimeWatermark, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, EventTimeWatermark, LogicalPlan, MapGroupsWithState} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} @@ -313,6 +313,21 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } + object MapGroupsWithStateStrategy extends Strategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case MapGroupsWithState( + func, keyDeser, valueDeser, groupAttr, dataAttr, outputAttr, + stateDeser, stateSer, child) => + val execPlan = MapGroupsWithStateExec( + func, keyDeser, valueDeser, + groupAttr, dataAttr, outputAttr, None, stateDeser, stateSer, + planLater(child)) + execPlan :: Nil + case _ => + Nil + } + } + // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { def numPartitions: Int = self.numPartitions @@ -354,6 +369,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.AppendColumnsWithObjectExec(f, childSer, newSer, planLater(child)) :: Nil case logical.MapGroups(f, key, value, grouping, data, objAttr, child) => execution.MapGroupsExec(f, key, value, grouping, data, objAttr, planLater(child)) :: Nil + case logical.MapGroupsWithState(f, key, value, grouping, data, output, _, _, child) => + execution.MapGroupsExec(f, key, value, grouping, data, output, planLater(child)) :: Nil case logical.CoGroup(f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, oAttr, left, right) => execution.CoGroupExec( f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, oAttr, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/StateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/StateImpl.scala new file mode 100644 index 0000000000000..c6c8d12c49083 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/StateImpl.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.State + +/** Internal implementation of the [[State]] interface */ +private[sql] class StateImpl[S](optionalValue: Option[S]) extends State[S] { + private var value: S = optionalValue.getOrElse(null.asInstanceOf[S]) + private var defined: Boolean = optionalValue.isDefined + private var updated: Boolean = false // whether value has been updated (but not removed) + private var removed: Boolean = false // whether value has eben removed + + // ========= Public API ========= + override def exists: Boolean = { + defined + } + + override def get(): S = { + if (defined) { + value + } else { + throw new NoSuchElementException("State is either not defined or has already been removed") + } + } + + override def update(newValue: S): Unit = { + value = newValue + defined = true + updated = true + removed = false + } + + override def remove(): Unit = { + defined = false + updated = false + removed = true + } + + // ========= Internal API ========= + + /** Whether the state has been marked for removing */ + def isRemoved: Boolean = { + removed + } + + /** Whether the state has been been updated */ + def isUpdated: Boolean = { + updated + } +} + +object StateImpl { + def apply[S](optionalValue: Option[S]): StateImpl[S] = new StateImpl[S](optionalValue) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index 4146bf3269a4b..717758fdf716f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -143,9 +143,15 @@ case class SimpleTypedAggregateExpression( override lazy val aggBufferAttributes: Seq[AttributeReference] = bufferSerializer.map(_.toAttribute.asInstanceOf[AttributeReference]) + private def serializeToBuffer(expr: Expression): Seq[Expression] = { + bufferSerializer.map(_.transform { + case _: BoundReference => expr + }) + } + override lazy val initialValues: Seq[Expression] = { val zero = Literal.fromObject(aggregator.zero, bufferExternalType) - bufferSerializer.map(ReferenceToExpressions(_, zero :: Nil)) + serializeToBuffer(zero) } override lazy val updateExpressions: Seq[Expression] = { @@ -154,8 +160,7 @@ case class SimpleTypedAggregateExpression( "reduce", bufferExternalType, bufferDeserializer :: inputDeserializer.get :: Nil) - - bufferSerializer.map(ReferenceToExpressions(_, reduced :: Nil)) + serializeToBuffer(reduced) } override lazy val mergeExpressions: Seq[Expression] = { @@ -170,8 +175,7 @@ case class SimpleTypedAggregateExpression( "merge", bufferExternalType, leftBuffer :: rightBuffer :: Nil) - - bufferSerializer.map(ReferenceToExpressions(_, merged :: Nil)) + serializeToBuffer(merged) } override lazy val evaluateExpression: Expression = { @@ -181,19 +185,17 @@ case class SimpleTypedAggregateExpression( outputExternalType, bufferDeserializer :: Nil) + val outputSerializeExprs = outputSerializer.map(_.transform { + case _: BoundReference => resultObj + }) + dataType match { - case s: StructType => + case _: StructType => val objRef = outputSerializer.head.find(_.isInstanceOf[BoundReference]).get - val struct = If( - IsNull(objRef), - Literal.create(null, dataType), - CreateStruct(outputSerializer)) - ReferenceToExpressions(struct, resultObj :: Nil) + If(IsNull(objRef), Literal.create(null, dataType), CreateStruct(outputSerializeExprs)) case _ => - assert(outputSerializer.length == 1) - outputSerializer.head transform { - case b: BoundReference => resultObj - } + assert(outputSerializeExprs.length == 1) + outputSerializeExprs.head } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index 1340c9bece38c..d024a3673d4ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -40,7 +40,8 @@ case class AnalyzeColumnCommand( val sessionState = sparkSession.sessionState val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db)) - val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdentWithDB)) + val relation = + EliminateSubqueryAliases(sparkSession.table(tableIdentWithDB).queryExecution.analyzed) // Compute total size val (catalogTable: CatalogTable, sizeInBytes: Long) = relation match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index 4a994e34aff85..30b6cc7617cb3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -41,7 +41,8 @@ case class AnalyzeTableCommand( val sessionState = sparkSession.sessionState val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db)) - val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdentWithDB)) + val relation = + EliminateSubqueryAliases(sparkSession.table(tableIdentWithDB).queryExecution.analyzed) relation match { case relation: CatalogRelation => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index dc8d97594c7a7..7afa4e78a3786 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -79,7 +79,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm // Queries all key-value pairs that are set in the SQLConf of the sparkSession. case None => val runFunc = (sparkSession: SparkSession) => { - sparkSession.conf.getAll.map { case (k, v) => Row(k, v) }.toSeq + sparkSession.conf.getAll.toSeq.sorted.map { case (k, v) => Row(k, v) } } (keyValueOutput, runFunc) @@ -87,8 +87,9 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm // SQLConf of the sparkSession. case Some(("-v", None)) => val runFunc = (sparkSession: SparkSession) => { - sparkSession.sessionState.conf.getAllDefinedConfs.map { case (key, defaultValue, doc) => - Row(key, defaultValue, doc) + sparkSession.sessionState.conf.getAllDefinedConfs.sorted.map { + case (key, defaultValue, doc) => + Row(key, Option(defaultValue).getOrElse(""), doc) } } val schema = StructType( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 90aeebd9329a1..5abd579476504 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -60,7 +60,7 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo identifier = table.identifier.copy( database = Some( table.identifier.database.getOrElse(sessionState.catalog.getCurrentDatabase))), - tracksPartitionsInCatalog = sparkSession.sessionState.conf.manageFilesourcePartitions) + tracksPartitionsInCatalog = sessionState.conf.manageFilesourcePartitions) val dataSource: BaseRelation = DataSource( sparkSession = sparkSession, @@ -71,15 +71,6 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo options = table.storage.properties ++ pathOption, catalogTable = Some(tableWithDefaultOptions)).resolveRelation() - dataSource match { - case fs: HadoopFsRelation => - if (table.tableType == CatalogTableType.EXTERNAL && fs.location.rootPaths.isEmpty) { - throw new AnalysisException( - "Cannot create a file-based external data source table without path") - } - case _ => - } - val partitionColumnNames = if (table.schema.nonEmpty) { table.partitionColumnNames } else { @@ -98,7 +89,7 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo // partition provider hive, but no partitions in the metastore. The user has to call // `msck repair table` to populate the table partitions. tracksPartitionsInCatalog = partitionColumnNames.nonEmpty && - sparkSession.sessionState.conf.manageFilesourcePartitions) + sessionState.conf.manageFilesourcePartitions) // We will return Nil or throw exception at the beginning if the table already exists, so when // we reach here, the table should not exist and we should set `ignoreIfExists` to false. sessionState.catalog.createTable(newTable, ignoreIfExists = false) @@ -131,7 +122,6 @@ case class CreateDataSourceTableAsSelectCommand( override def run(sparkSession: SparkSession): Seq[Row] = { assert(table.tableType != CatalogTableType.VIEW) assert(table.provider.isDefined) - assert(table.schema.isEmpty) val sessionState = sparkSession.sessionState val db = table.identifier.database.getOrElse(sessionState.catalog.getCurrentDatabase) @@ -153,6 +143,8 @@ case class CreateDataSourceTableAsSelectCommand( saveDataIntoTable( sparkSession, table, table.storage.locationUri, query, mode, tableExists = true) } else { + assert(table.schema.isEmpty) + val tableLocation = if (table.tableType == CatalogTableType.MANAGED) { Some(sessionState.catalog.defaultTablePath(table.identifier)) } else { @@ -172,8 +164,7 @@ case class CreateDataSourceTableAsSelectCommand( case fs: HadoopFsRelation if table.partitionColumnNames.nonEmpty && sparkSession.sqlContext.conf.manageFilesourcePartitions => // Need to recover partitions into the metastore so our saved data is visible. - sparkSession.sessionState.executePlan( - AlterTableRecoverPartitionsCommand(table.identifier)).toRdd + sessionState.executePlan(AlterTableRecoverPartitionsCommand(table.identifier)).toRdd case _ => } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 246894813c3b9..bb903a2662369 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -450,7 +450,7 @@ case class DescribeTableCommand( if (metadata.schema.isEmpty) { // In older version(prior to 2.1) of Spark, the table schema can be empty and should be // inferred at runtime. We should still support it. - describeSchema(catalog.lookupRelation(metadata.identifier).schema, result) + describeSchema(sparkSession.table(metadata.identifier).schema, result) } else { describeSchema(metadata.schema, result) } @@ -528,8 +528,10 @@ case class DescribeTableCommand( private def describeViewInfo(metadata: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { append(buffer, "", "", "") append(buffer, "# View Information", "", "") - append(buffer, "View Original Text:", metadata.viewOriginalText.getOrElse(""), "") - append(buffer, "View Expanded Text:", metadata.viewText.getOrElse(""), "") + append(buffer, "View Text:", metadata.viewText.getOrElse(""), "") + append(buffer, "View Default Database:", metadata.viewDefaultDatabase.getOrElse(""), "") + append(buffer, "View Query Output Columns:", + metadata.viewQueryColumnNames.mkString("[", ", ", "]"), "") } private def describeBucketingInfo(metadata: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 154141bf83c7d..921c84895598c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -17,10 +17,10 @@ package org.apache.spark.sql.execution.command -import scala.util.control.NonFatal +import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Row, SparkSession} -import org.apache.spark.sql.catalyst.{SQLBuilder, TableIdentifier} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.Alias @@ -47,8 +47,8 @@ object LocalTempView extends ViewType /** * GlobalTempView means cross-session global temporary views. Its lifetime is the lifetime of the * Spark application, i.e. it will be automatically dropped when the application terminates. It's - * tied to a system preserved database `_global_temp`, and we must use the qualified name to refer a - * global temp view, e.g. SELECT * FROM _global_temp.view1. + * tied to a system preserved database `global_temp`, and we must use the qualified name to refer a + * global temp view, e.g. SELECT * FROM global_temp.view1. */ object GlobalTempView extends ViewType @@ -64,9 +64,9 @@ object PersistedView extends ViewType /** - * Create or replace a view with given query plan. This command will convert the query plan to - * canonicalized SQL string, and store it as view text in metastore, if we need to create a - * permanent view. + * Create or replace a view with given query plan. This command will generate some view-specific + * properties(e.g. view default database, view query output column names) and store them as + * properties in metastore, if we need to create a permanent view. * * @param name the name of this view. * @param userSpecifiedColumns the output column names and optional comments specified by users, @@ -75,8 +75,8 @@ object PersistedView extends ViewType * @param properties the properties of this view. * @param originalText the original SQL text of this view, can be None if this view is created via * Dataset API. - * @param child the logical plan that represents the view; this is used to generate a canonicalized - * version of the SQL that can be saved in the catalog. + * @param child the logical plan that represents the view; this is used to generate the logical + * plan for temporary view and the view schema. * @param allowExisting if true, and if the view already exists, noop; if false, and if the view * already exists, throws analysis exception. * @param replace if true, and if the view already exists, updates it; if false, and if the view @@ -95,6 +95,8 @@ case class CreateViewCommand( viewType: ViewType) extends RunnableCommand { + import ViewHelper._ + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) if (viewType == PersistedView) { @@ -137,22 +139,12 @@ case class CreateViewCommand( // This should be called after `qe.assertAnalyzed()` (i.e., `child` can be resolved) verifyTemporaryObjectsNotExists(sparkSession) - val aliasedPlan = if (userSpecifiedColumns.isEmpty) { - analyzedPlan - } else { - val projectList = analyzedPlan.output.zip(userSpecifiedColumns).map { - case (attr, (colName, None)) => Alias(attr, colName)() - case (attr, (colName, Some(colComment))) => - val meta = new MetadataBuilder().putString("comment", colComment).build() - Alias(attr, colName)(explicitMetadata = Some(meta)) - } - sparkSession.sessionState.executePlan(Project(projectList, analyzedPlan)).analyzed - } - val catalog = sparkSession.sessionState.catalog if (viewType == LocalTempView) { + val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) catalog.createTempView(name.table, aliasedPlan, overrideIfExists = replace) } else if (viewType == GlobalTempView) { + val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) catalog.createGlobalTempView(name.table, aliasedPlan, overrideIfExists = replace) } else if (catalog.tableExists(name)) { val tableMetadata = catalog.getTableMetadata(name) @@ -163,7 +155,7 @@ case class CreateViewCommand( throw new AnalysisException(s"$name is not a view") } else if (replace) { // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` - catalog.alterTable(prepareTable(sparkSession, aliasedPlan)) + catalog.alterTable(prepareTable(sparkSession, analyzedPlan)) } else { // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already // exists. @@ -173,7 +165,7 @@ case class CreateViewCommand( } } else { // Create the view if it doesn't exist. - catalog.createTable(prepareTable(sparkSession, aliasedPlan), ignoreIfExists = false) + catalog.createTable(prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) } Seq.empty[Row] } @@ -207,29 +199,43 @@ case class CreateViewCommand( } /** - * Returns a [[CatalogTable]] that can be used to save in the catalog. This comment canonicalize - * SQL based on the analyzed plan, and also creates the proper schema for the view. + * If `userSpecifiedColumns` is defined, alias the analyzed plan to the user specified columns, + * else return the analyzed plan directly. */ - private def prepareTable(sparkSession: SparkSession, aliasedPlan: LogicalPlan): CatalogTable = { - val viewSQL: String = new SQLBuilder(aliasedPlan).toSQL - - // Validate the view SQL - make sure we can parse it and analyze it. - // If we cannot analyze the generated query, there is probably a bug in SQL generation. - try { - sparkSession.sql(viewSQL).queryExecution.assertAnalyzed() - } catch { - case NonFatal(e) => - throw new RuntimeException(s"Failed to analyze the canonicalized SQL: $viewSQL", e) + private def aliasPlan(session: SparkSession, analyzedPlan: LogicalPlan): LogicalPlan = { + if (userSpecifiedColumns.isEmpty) { + analyzedPlan + } else { + val projectList = analyzedPlan.output.zip(userSpecifiedColumns).map { + case (attr, (colName, None)) => Alias(attr, colName)() + case (attr, (colName, Some(colComment))) => + val meta = new MetadataBuilder().putString("comment", colComment).build() + Alias(attr, colName)(explicitMetadata = Some(meta)) + } + session.sessionState.executePlan(Project(projectList, analyzedPlan)).analyzed } + } + + /** + * Returns a [[CatalogTable]] that can be used to save in the catalog. Generate the view-specific + * properties(e.g. view default database, view query output column names) and store them as + * properties in the CatalogTable, and also creates the proper schema for the view. + */ + private def prepareTable(session: SparkSession, analyzedPlan: LogicalPlan): CatalogTable = { + if (originalText.isEmpty) { + throw new AnalysisException( + "It is not allowed to create a persisted view from the Dataset API") + } + + val newProperties = generateViewProperties(properties, session, analyzedPlan) CatalogTable( identifier = name, tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, - schema = aliasedPlan.schema, - properties = properties, - viewOriginalText = originalText, - viewText = Some(viewSQL), + schema = aliasPlan(session, analyzedPlan).schema, + properties = newProperties, + viewText = originalText, comment = comment ) } @@ -244,14 +250,16 @@ case class CreateViewCommand( * @param name the name of this view. * @param originalText the original SQL text of this view. Note that we can only alter a view by * SQL API, which means we always have originalText. - * @param query the logical plan that represents the view; this is used to generate a canonicalized - * version of the SQL that can be saved in the catalog. + * @param query the logical plan that represents the view; this is used to generate the new view + * schema. */ case class AlterViewAsCommand( name: TableIdentifier, originalText: String, query: LogicalPlan) extends RunnableCommand { + import ViewHelper._ + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query) override def run(session: SparkSession): Seq[Row] = { @@ -275,21 +283,79 @@ case class AlterViewAsCommand( throw new AnalysisException(s"${viewMeta.identifier} is not a view.") } - val viewSQL: String = new SQLBuilder(analyzedPlan).toSQL - // Validate the view SQL - make sure we can parse it and analyze it. - // If we cannot analyze the generated query, there is probably a bug in SQL generation. - try { - session.sql(viewSQL).queryExecution.assertAnalyzed() - } catch { - case NonFatal(e) => - throw new RuntimeException(s"Failed to analyze the canonicalized SQL: $viewSQL", e) - } + val newProperties = generateViewProperties(viewMeta.properties, session, analyzedPlan) val updatedViewMeta = viewMeta.copy( schema = analyzedPlan.schema, - viewOriginalText = Some(originalText), - viewText = Some(viewSQL)) + properties = newProperties, + viewText = Some(originalText)) session.sessionState.catalog.alterTable(updatedViewMeta) } } + +object ViewHelper { + + import CatalogTable._ + + /** + * Generate the view default database in `properties`. + */ + private def generateViewDefaultDatabase(databaseName: String): Map[String, String] = { + Map(VIEW_DEFAULT_DATABASE -> databaseName) + } + + /** + * Generate the view query output column names in `properties`. + */ + private def generateQueryColumnNames(columns: Seq[String]): Map[String, String] = { + val props = new mutable.HashMap[String, String] + if (columns.nonEmpty) { + props.put(VIEW_QUERY_OUTPUT_NUM_COLUMNS, columns.length.toString) + columns.zipWithIndex.foreach { case (colName, index) => + props.put(s"$VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX$index", colName) + } + } + props.toMap + } + + /** + * Remove the view query output column names in `properties`. + */ + private def removeQueryColumnNames(properties: Map[String, String]): Map[String, String] = { + // We can't use `filterKeys` here, as the map returned by `filterKeys` is not serializable, + // while `CatalogTable` should be serializable. + properties.filterNot { case (key, _) => + key.startsWith(VIEW_QUERY_OUTPUT_PREFIX) + } + } + + /** + * Generate the view properties in CatalogTable, including: + * 1. view default database that is used to provide the default database name on view resolution. + * 2. the output column names of the query that creates a view, this is used to map the output of + * the view child to the view output during view resolution. + * + * @param properties the `properties` in CatalogTable. + * @param session the spark session. + * @param analyzedPlan the analyzed logical plan that represents the child of a view. + * @return new view properties including view default database and query column names properties. + */ + def generateViewProperties( + properties: Map[String, String], + session: SparkSession, + analyzedPlan: LogicalPlan): Map[String, String] = { + // Generate the query column names, throw an AnalysisException if there exists duplicate column + // names. + val queryOutput = analyzedPlan.schema.fieldNames + assert(queryOutput.distinct.size == queryOutput.size, + s"The view output ${queryOutput.mkString("(", ",", ")")} contains duplicate column name.") + + // Generate the view default database name. + val viewDefaultDatabase = session.sessionState.catalog.getCurrentDatabase + + removeQueryColumnNames(properties) ++ + generateViewDefaultDatabase(viewDefaultDatabase) ++ + generateQueryColumnNames(queryOutput) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 29afe5751bc16..ecfcafe69c1f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -104,20 +104,12 @@ case class DataSource( * dataTypes in `userSpecifiedSchema`. All subsequent triggers for this stream will re-use * this information, therefore calls to this method should be very cheap, i.e. there won't * be any further inference in any triggers. - * 4. `df.saveAsTable(tableThatExisted)`: In this case, we call this method to resolve the - * existing table's partitioning scheme. This is achieved by not providing - * `userSpecifiedSchema`. For this case, we add the boolean `justPartitioning` for an early - * exit, if we don't care about the schema of the original table. * * @param format the file format object for this DataSource - * @param justPartitioning Whether to exit early and provide just the schema partitioning. * @return A pair of the data schema (excluding partition columns) and the schema of the partition - * columns. If `justPartitioning` is `true`, then the dataSchema will be provided as - * `null`. + * columns. */ - private def getOrInferFileFormatSchema( - format: FileFormat, - justPartitioning: Boolean = false): (StructType, StructType) = { + private def getOrInferFileFormatSchema(format: FileFormat): (StructType, StructType) = { // the operations below are expensive therefore try not to do them if we don't need to, e.g., // in streaming mode, we have already inferred and registered partition columns, we will // never have to materialize the lazy val below @@ -174,9 +166,7 @@ case class DataSource( StructType(partitionFields) } } - if (justPartitioning) { - return (null, partitionSchema) - } + val dataSchema = userSpecifiedSchema.map { schema => val equality = sparkSession.sessionState.conf.resolver StructType(schema.filterNot(f => partitionSchema.exists(p => equality(p.name, f.name)))) @@ -434,26 +424,6 @@ case class DataSource( val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis PartitioningUtils.validatePartitionColumn(data.schema, partitionColumns, caseSensitive) - // If we are appending to a table that already exists, make sure the partitioning matches - // up. If we fail to load the table for whatever reason, ignore the check. - if (mode == SaveMode.Append) { - val existingPartitionColumns = Try { - getOrInferFileFormatSchema(format, justPartitioning = true)._2.fieldNames.toList - }.getOrElse(Seq.empty[String]) - // TODO: Case sensitivity. - val sameColumns = - existingPartitionColumns.map(_.toLowerCase()) == partitionColumns.map(_.toLowerCase()) - if (existingPartitionColumns.nonEmpty && !sameColumns) { - throw new AnalysisException( - s"""Requested partitioning does not match existing partitioning. - |Existing partitioning columns: - | ${existingPartitionColumns.mkString(", ")} - |Requested partitioning columns: - | ${partitionColumns.mkString(", ")} - |""".stripMargin) - } - } - // SPARK-17230: Resolve the partition columns so InsertIntoHadoopFsRelationCommand does // not need to have the query as child, to avoid to analyze an optimized query, // because InsertIntoHadoopFsRelationCommand will be optimized first. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 3d3db06eee0c6..19db293132f54 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -17,18 +17,17 @@ package org.apache.spark.sql.execution.datasources -import scala.collection.mutable.ArrayBuffer +import java.util.concurrent.Callable -import org.apache.hadoop.fs.Path +import scala.collection.mutable.ArrayBuffer import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, InternalRow, QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTablePartition, SimpleCatalogRelation} -import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SimpleCatalogRelation} import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation @@ -37,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, UnknownPa import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -44,6 +44,8 @@ import org.apache.spark.unsafe.types.UTF8String /** * Replaces generic operations with specific variants that are designed to work with Spark * SQL Data Sources. + * + * Note that, this rule must be run after [[PreprocessTableInsertion]]. */ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { @@ -127,30 +129,9 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { projectList } - /** - * Returns true if the [[InsertIntoTable]] plan has already been preprocessed by analyzer rule - * [[PreprocessTableInsertion]]. It is important that this rule([[DataSourceAnalysis]]) has to - * be run after [[PreprocessTableInsertion]], to normalize the column names in partition spec and - * fix the schema mismatch by adding Cast. - */ - private def hasBeenPreprocessed( - tableOutput: Seq[Attribute], - partSchema: StructType, - partSpec: Map[String, Option[String]], - query: LogicalPlan): Boolean = { - val partColNames = partSchema.map(_.name).toSet - query.resolved && partSpec.keys.forall(partColNames.contains) && { - val staticPartCols = partSpec.filter(_._2.isDefined).keySet - val expectedColumns = tableOutput.filterNot(a => staticPartCols.contains(a.name)) - expectedColumns.toStructType.sameType(query.schema) - } - } - override def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoTable( - l @ LogicalRelation(t: HadoopFsRelation, _, table), parts, query, overwrite, false) - if hasBeenPreprocessed(l.output, t.partitionSchema, parts, query) => - + l @ LogicalRelation(t: HadoopFsRelation, _, table), parts, query, overwrite, false) => // If the InsertIntoTable command is for a partitioned HadoopFsRelation and // the user has specified static partitions, we add a Project operator on top of the query // to include those constant column values in the query result. @@ -215,37 +196,43 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { /** - * Replaces [[SimpleCatalogRelation]] with data source table if its table property contains data - * source information. + * Replaces [[SimpleCatalogRelation]] with data source table if its table provider is not hive. */ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] { - private def readDataSourceTable( - sparkSession: SparkSession, - simpleCatalogRelation: SimpleCatalogRelation): LogicalPlan = { - val table = simpleCatalogRelation.catalogTable - val pathOption = table.storage.locationUri.map("path" -> _) - val dataSource = - DataSource( - sparkSession, - userSpecifiedSchema = Some(table.schema), - partitionColumns = table.partitionColumnNames, - bucketSpec = table.bucketSpec, - className = table.provider.get, - options = table.storage.properties ++ pathOption) - - LogicalRelation( - dataSource.resolveRelation(), - expectedOutputAttributes = Some(simpleCatalogRelation.output), - catalogTable = Some(table)) + private def readDataSourceTable(table: CatalogTable): LogicalPlan = { + val qualifiedTableName = QualifiedTableName(table.database, table.identifier.table) + val cache = sparkSession.sessionState.catalog.tableRelationCache + val withHiveSupport = + sparkSession.sparkContext.conf.get(StaticSQLConf.CATALOG_IMPLEMENTATION) == "hive" + + cache.get(qualifiedTableName, new Callable[LogicalPlan]() { + override def call(): LogicalPlan = { + val pathOption = table.storage.locationUri.map("path" -> _) + val dataSource = + DataSource( + sparkSession, + // In older version(prior to 2.1) of Spark, the table schema can be empty and should be + // inferred at runtime. We should still support it. + userSpecifiedSchema = if (table.schema.isEmpty) None else Some(table.schema), + partitionColumns = table.partitionColumnNames, + bucketSpec = table.bucketSpec, + className = table.provider.get, + options = table.storage.properties ++ pathOption, + // TODO: improve `InMemoryCatalog` and remove this limitation. + catalogTable = if (withHiveSupport) Some(table) else None) + + LogicalRelation(dataSource.resolveRelation(), catalogTable = Some(table)) + } + }) } override def apply(plan: LogicalPlan): LogicalPlan = plan transform { case i @ InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) if DDLUtils.isDatasourceTable(s.metadata) => - i.copy(table = readDataSourceTable(sparkSession, s)) + i.copy(table = readDataSourceTable(s.metadata)) case s: SimpleCatalogRelation if DDLUtils.isDatasourceTable(s.metadata) => - readDataSourceTable(sparkSession, s) + readDataSourceTable(s.metadata) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 16c5193eda8df..be13cbc51a9d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{QueryExecution, SQLExecution, UnsafeKVExternalSorter} import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -311,10 +312,11 @@ object FileFormatWriter extends Logging { /** Expressions that given a partition key build a string like: col1=val/col2=val/... */ private def partitionStringExpression: Seq[Expression] = { description.partitionColumns.zipWithIndex.flatMap { case (c, i) => + // TODO: use correct timezone for partition values. val escaped = ScalaUDF( ExternalCatalogUtils.escapePathName _, StringType, - Seq(Cast(c, StringType)), + Seq(Cast(c, StringType, Option(DateTimeUtils.defaultTimeZone().getID))), Seq(StringType)) val str = If(IsNull(c), Literal(ExternalCatalogUtils.DEFAULT_PARTITION_NAME), escaped) val partitionName = Literal(c.name + "=") :: str :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 6d0671d4cb16f..26e1380eca499 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -62,7 +62,7 @@ object FileSourceStrategy extends Strategy with Logging { val filterSet = ExpressionSet(filters) // The attribute name of predicate could be different than the one in schema in case of - // case insensitive, we should change them to match the one in schema, so we donot need to + // case insensitive, we should change them to match the one in schema, so we do not need to // worry about case sensitivity anymore. val normalizedFilters = filters.map { e => e transform { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index 82c1599a39df6..75f87a5503b8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -30,6 +30,7 @@ import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.SerializableConfiguration @@ -135,9 +136,11 @@ abstract class PartitioningAwareFileIndex( // we need to cast into the data type that user specified. def castPartitionValuesToUserSchema(row: InternalRow) = { InternalRow((0 until row.numFields).map { i => + // TODO: use correct timezone for partition values. Cast( Literal.create(row.getUTF8String(i), StringType), - userProvidedSchema.fields(i).dataType).eval() + userProvidedSchema.fields(i).dataType, + Option(DateTimeUtils.defaultTimeZone().getID)).eval() }: _*) } @@ -385,55 +388,54 @@ object PartitioningAwareFileIndex extends Logging { logTrace(s"Listing $path") val fs = path.getFileSystem(hadoopConf) val name = path.getName.toLowerCase - if (shouldFilterOut(name)) { - Seq.empty[FileStatus] - } else { - // [SPARK-17599] Prevent InMemoryFileIndex from failing if path doesn't exist - // Note that statuses only include FileStatus for the files and dirs directly under path, - // and does not include anything else recursively. - val statuses = try fs.listStatus(path) catch { - case _: FileNotFoundException => - logWarning(s"The directory $path was not found. Was it deleted very recently?") - Array.empty[FileStatus] - } - val allLeafStatuses = { - val (dirs, topLevelFiles) = statuses.partition(_.isDirectory) - val nestedFiles: Seq[FileStatus] = sessionOpt match { - case Some(session) => - bulkListLeafFiles(dirs.map(_.getPath), hadoopConf, filter, session).flatMap(_._2) - case _ => - dirs.flatMap(dir => listLeafFiles(dir.getPath, hadoopConf, filter, sessionOpt)) - } - val allFiles = topLevelFiles ++ nestedFiles - if (filter != null) allFiles.filter(f => filter.accept(f.getPath)) else allFiles - } + // [SPARK-17599] Prevent InMemoryFileIndex from failing if path doesn't exist + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses = try fs.listStatus(path) catch { + case _: FileNotFoundException => + logWarning(s"The directory $path was not found. Was it deleted very recently?") + Array.empty[FileStatus] + } - allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { - case f: LocatedFileStatus => - f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not - // be a big deal since we always use to `listLeafFilesInParallel` when the number of - // paths exceeds threshold. - case f => - // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), - // which is very slow on some file system (RawLocalFileSystem, which is launch a - // subprocess and parse the stdout). - val locations = fs.getFileBlockLocations(f, 0, f.getLen) - val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, - f.getModificationTime, 0, null, null, null, null, f.getPath, locations) - if (f.isSymlink) { - lfs.setSymlink(f.getSymlink) - } - lfs + val filteredStatuses = statuses.filterNot(status => shouldFilterOut(status.getPath.getName)) + + val allLeafStatuses = { + val (dirs, topLevelFiles) = filteredStatuses.partition(_.isDirectory) + val nestedFiles: Seq[FileStatus] = sessionOpt match { + case Some(session) => + bulkListLeafFiles(dirs.map(_.getPath), hadoopConf, filter, session).flatMap(_._2) + case _ => + dirs.flatMap(dir => listLeafFiles(dir.getPath, hadoopConf, filter, sessionOpt)) } + val allFiles = topLevelFiles ++ nestedFiles + if (filter != null) allFiles.filter(f => filter.accept(f.getPath)) else allFiles + } + + allLeafStatuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { + case f: LocatedFileStatus => + f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `listLeafFilesInParallel` when the number of + // paths exceeds threshold. + case f => + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), + // which is very slow on some file system (RawLocalFileSystem, which is launch a + // subprocess and parse the stdout). + val locations = fs.getFileBlockLocations(f, 0, f.getLen) + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + lfs } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index b0feaeb84e9f4..38970160d5fb3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -19,13 +19,15 @@ package org.apache.spark.sql.execution.datasources.csv import java.nio.charset.{Charset, StandardCharsets} +import com.univocity.parsers.csv.CsvParser import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce._ import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Dataset, Encoders, SparkSession} import org.apache.spark.sql.catalyst.InternalRow @@ -60,7 +62,7 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { val paths = files.map(_.getPath.toString) val lines: Dataset[String] = readText(sparkSession, csvOptions, paths) val firstLine: String = findFirstLine(csvOptions, lines) - val firstRow = new CsvReader(csvOptions).parseLine(firstLine) + val firstRow = new CsvParser(csvOptions.asParserSettings).parseLine(firstLine) val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis val header = makeSafeHeader(firstRow, csvOptions, caseSensitive) @@ -130,7 +132,18 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { CompressionCodecs.setCodecConfiguration(conf, codec) } - new CSVOutputWriterFactory(csvOptions) + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new CsvOutputWriter(path, dataSchema, context, csvOptions) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + ".csv" + CodecStreams.getCompressionExtension(context) + } + } } override def buildReader( @@ -143,7 +156,6 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { val csvOptions = new CSVOptions(options) val commentPrefix = csvOptions.comment.toString - val headers = requiredSchema.fields.map(_.name) val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -158,23 +170,15 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { } } + // Consumes the header in the iterator. CSVRelation.dropHeaderLine(file, lineIterator, csvOptions) - val csvParser = new CsvReader(csvOptions) - val tokenizedIterator = lineIterator.filter { line => + val filteredIter = lineIterator.filter { line => line.trim.nonEmpty && !line.startsWith(commentPrefix) - }.map { line => - csvParser.parseLine(line) - } - val parser = CSVRelation.csvParser(dataSchema, requiredSchema.fieldNames, csvOptions) - var numMalformedRecords = 0 - tokenizedIterator.flatMap { recordTokens => - val row = parser(recordTokens, numMalformedRecords) - if (row.isEmpty) { - numMalformedRecords += 1 - } - row } + + val parser = new UnivocityParser(dataSchema, requiredSchema, csvOptions) + filteredIter.flatMap(parser.parse) } } @@ -228,3 +232,18 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { schema.foreach(field => verifyType(field.dataType)) } } + +private[csv] class CsvOutputWriter( + path: String, + dataSchema: StructType, + context: TaskAttemptContext, + params: CSVOptions) extends OutputWriter with Logging { + + private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path)) + + private val gen = new UnivocityGenerator(dataSchema, writer, params) + + override def write(row: InternalRow): Unit = gen.write(row) + + override def close(): Unit = gen.close() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index adc92fe5a31e6..065bf53574366 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -217,124 +217,6 @@ private[csv] object CSVInferSchema { } private[csv] object CSVTypeCast { - // A `ValueConverter` is responsible for converting the given value to a desired type. - private type ValueConverter = String => Any - - /** - * Create converters which cast each given string datum to each specified type in given schema. - * Currently, we do not support complex types (`ArrayType`, `MapType`, `StructType`). - * - * For string types, this is simply the datum. - * For other types, this is converted into the value according to the type. - * For other nullable types, returns null if it is null or equals to the value specified - * in `nullValue` option. - * - * @param schema schema that contains data types to cast the given value into. - * @param options CSV options. - */ - def makeConverters( - schema: StructType, - options: CSVOptions = CSVOptions()): Array[ValueConverter] = { - schema.map(f => makeConverter(f.name, f.dataType, f.nullable, options)).toArray - } - - /** - * Create a converter which converts the string value to a value according to a desired type. - */ - def makeConverter( - name: String, - dataType: DataType, - nullable: Boolean = true, - options: CSVOptions = CSVOptions()): ValueConverter = dataType match { - case _: ByteType => (d: String) => - nullSafeDatum(d, name, nullable, options)(_.toByte) - - case _: ShortType => (d: String) => - nullSafeDatum(d, name, nullable, options)(_.toShort) - - case _: IntegerType => (d: String) => - nullSafeDatum(d, name, nullable, options)(_.toInt) - - case _: LongType => (d: String) => - nullSafeDatum(d, name, nullable, options)(_.toLong) - - case _: FloatType => (d: String) => - nullSafeDatum(d, name, nullable, options) { - case options.nanValue => Float.NaN - case options.negativeInf => Float.NegativeInfinity - case options.positiveInf => Float.PositiveInfinity - case datum => - Try(datum.toFloat) - .getOrElse(NumberFormat.getInstance(Locale.US).parse(datum).floatValue()) - } - - case _: DoubleType => (d: String) => - nullSafeDatum(d, name, nullable, options) { - case options.nanValue => Double.NaN - case options.negativeInf => Double.NegativeInfinity - case options.positiveInf => Double.PositiveInfinity - case datum => - Try(datum.toDouble) - .getOrElse(NumberFormat.getInstance(Locale.US).parse(datum).doubleValue()) - } - - case _: BooleanType => (d: String) => - nullSafeDatum(d, name, nullable, options)(_.toBoolean) - - case dt: DecimalType => (d: String) => - nullSafeDatum(d, name, nullable, options) { datum => - val value = new BigDecimal(datum.replaceAll(",", "")) - Decimal(value, dt.precision, dt.scale) - } - - case _: TimestampType => (d: String) => - nullSafeDatum(d, name, nullable, options) { datum => - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681. - Try(options.timestampFormat.parse(datum).getTime * 1000L) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.stringToTime(datum).getTime * 1000L - } - } - - case _: DateType => (d: String) => - nullSafeDatum(d, name, nullable, options) { datum => - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681.x - Try(DateTimeUtils.millisToDays(options.dateFormat.parse(datum).getTime)) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime) - } - } - - case _: StringType => (d: String) => - nullSafeDatum(d, name, nullable, options)(UTF8String.fromString(_)) - - case udt: UserDefinedType[_] => (datum: String) => - makeConverter(name, udt.sqlType, nullable, options) - - case _ => throw new RuntimeException(s"Unsupported type: ${dataType.typeName}") - } - - private def nullSafeDatum( - datum: String, - name: String, - nullable: Boolean, - options: CSVOptions)(converter: ValueConverter): Any = { - if (datum == options.nullValue || datum == null) { - if (!nullable) { - throw new RuntimeException(s"null value found but field $name is not nullable.") - } - null - } else { - converter.apply(datum) - } - } - /** * Helper method that converts string representation of a character to actual character. * It handles some Java escaped strings and throws exception if given string is longer than one diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 21e50307b5ab0..140ce23958dc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.csv import java.nio.charset.StandardCharsets import java.util.Locale +import com.univocity.parsers.csv.{CsvParserSettings, CsvWriterSettings, UnescapedQuoteHandling} import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.internal.Logging @@ -126,6 +127,39 @@ private[csv] class CSVOptions(@transient private val parameters: CaseInsensitive val inputBufferSize = 128 val isCommentSet = this.comment != '\u0000' + + def asWriterSettings: CsvWriterSettings = { + val writerSettings = new CsvWriterSettings() + val format = writerSettings.getFormat + format.setDelimiter(delimiter) + format.setQuote(quote) + format.setQuoteEscape(escape) + format.setComment(comment) + writerSettings.setNullValue(nullValue) + writerSettings.setEmptyValue(nullValue) + writerSettings.setSkipEmptyLines(true) + writerSettings.setQuoteAllFields(quoteAll) + writerSettings.setQuoteEscapingEnabled(escapeQuotes) + writerSettings + } + + def asParserSettings: CsvParserSettings = { + val settings = new CsvParserSettings() + val format = settings.getFormat + format.setDelimiter(delimiter) + format.setQuote(quote) + format.setQuoteEscape(escape) + format.setComment(comment) + settings.setIgnoreLeadingWhitespaces(ignoreLeadingWhiteSpaceFlag) + settings.setIgnoreTrailingWhitespaces(ignoreTrailingWhiteSpaceFlag) + settings.setReadInputOnSeparateThread(false) + settings.setInputBufferSize(inputBufferSize) + settings.setMaxColumns(maxColumns) + settings.setNullValue(nullValue) + settings.setMaxCharsPerColumn(maxCharsPerColumn) + settings.setUnescapedQuoteHandling(UnescapedQuoteHandling.STOP_AT_DELIMITER) + settings + } } object CSVOptions { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala deleted file mode 100644 index 6239508ec9422..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.csv - -import java.io.{CharArrayWriter, OutputStream, StringReader} -import java.nio.charset.StandardCharsets - -import com.univocity.parsers.csv._ - -import org.apache.spark.internal.Logging - -/** - * Read and parse CSV-like input - * - * @param params Parameters object - */ -private[csv] class CsvReader(params: CSVOptions) { - - private val parser: CsvParser = { - val settings = new CsvParserSettings() - val format = settings.getFormat - format.setDelimiter(params.delimiter) - format.setQuote(params.quote) - format.setQuoteEscape(params.escape) - format.setComment(params.comment) - settings.setIgnoreLeadingWhitespaces(params.ignoreLeadingWhiteSpaceFlag) - settings.setIgnoreTrailingWhitespaces(params.ignoreTrailingWhiteSpaceFlag) - settings.setReadInputOnSeparateThread(false) - settings.setInputBufferSize(params.inputBufferSize) - settings.setMaxColumns(params.maxColumns) - settings.setNullValue(params.nullValue) - settings.setMaxCharsPerColumn(params.maxCharsPerColumn) - settings.setUnescapedQuoteHandling(UnescapedQuoteHandling.STOP_AT_DELIMITER) - - new CsvParser(settings) - } - - /** - * parse a line - * - * @param line a String with no newline at the end - * @return array of strings where each string is a field in the CSV record - */ - def parseLine(line: String): Array[String] = parser.parseLine(line) -} - -/** - * Converts a sequence of string to CSV string - * - * @param params Parameters object for configuration - * @param headers headers for columns - */ -private[csv] class LineCsvWriter( - params: CSVOptions, - headers: Seq[String], - output: OutputStream) extends Logging { - private val writerSettings = new CsvWriterSettings - private val format = writerSettings.getFormat - - format.setDelimiter(params.delimiter) - format.setQuote(params.quote) - format.setQuoteEscape(params.escape) - format.setComment(params.comment) - - writerSettings.setNullValue(params.nullValue) - writerSettings.setEmptyValue(params.nullValue) - writerSettings.setSkipEmptyLines(true) - writerSettings.setQuoteAllFields(params.quoteAll) - writerSettings.setHeaders(headers: _*) - writerSettings.setQuoteEscapingEnabled(params.escapeQuotes) - - private val writer = new CsvWriter(output, StandardCharsets.UTF_8, writerSettings) - - def writeRow(row: Seq[String], includeHeader: Boolean): Unit = { - if (includeHeader) { - writer.writeHeaders() - } - - writer.writeRow(row: _*) - } - - def close(): Unit = { - writer.close() - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 8c19be48c2118..19058c23abe75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -17,19 +17,12 @@ package org.apache.spark.sql.execution.datasources.csv -import scala.util.control.NonFatal - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.TaskAttemptContext +import com.univocity.parsers.csv.CsvParser import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter, OutputWriterFactory, PartitionedFile} -import org.apache.spark.sql.types._ +import org.apache.spark.sql.execution.datasources.PartitionedFile object CSVRelation extends Logging { @@ -40,7 +33,7 @@ object CSVRelation extends Logging { // If header is set, make sure firstLine is materialized before sending to executors. val commentPrefix = params.comment.toString file.rdd.mapPartitions { iter => - val parser = new CsvReader(params) + val parser = new CsvParser(params.asParserSettings) val filteredIter = iter.filter { line => line.trim.nonEmpty && !line.startsWith(commentPrefix) } @@ -56,91 +49,6 @@ object CSVRelation extends Logging { } } - /** - * Returns a function that parses a single CSV record (in the form of an array of strings in which - * each element represents a column) and turns it into either one resulting row or no row (if the - * the record is malformed). - * - * The 2nd argument in the returned function represents the total number of malformed rows - * observed so far. - */ - // This is pretty convoluted and we should probably rewrite the entire CSV parsing soon. - def csvParser( - schema: StructType, - requiredColumns: Array[String], - params: CSVOptions): (Array[String], Int) => Option[InternalRow] = { - val requiredFields = StructType(requiredColumns.map(schema(_))).fields - val safeRequiredFields = if (params.dropMalformed) { - // If `dropMalformed` is enabled, then it needs to parse all the values - // so that we can decide which row is malformed. - requiredFields ++ schema.filterNot(requiredFields.contains(_)) - } else { - requiredFields - } - val safeRequiredIndices = new Array[Int](safeRequiredFields.length) - schema.zipWithIndex.filter { case (field, _) => - safeRequiredFields.contains(field) - }.foreach { case (field, index) => - safeRequiredIndices(safeRequiredFields.indexOf(field)) = index - } - val requiredSize = requiredFields.length - val row = new GenericInternalRow(requiredSize) - val converters = CSVTypeCast.makeConverters(schema, params) - - (tokens: Array[String], numMalformedRows) => { - if (params.dropMalformed && schema.length != tokens.length) { - if (numMalformedRows < params.maxMalformedLogPerPartition) { - logWarning(s"Dropping malformed line: ${tokens.mkString(params.delimiter.toString)}") - } - if (numMalformedRows == params.maxMalformedLogPerPartition - 1) { - logWarning( - s"More than ${params.maxMalformedLogPerPartition} malformed records have been " + - "found on this partition. Malformed records from now on will not be logged.") - } - None - } else if (params.failFast && schema.length != tokens.length) { - throw new RuntimeException(s"Malformed line in FAILFAST mode: " + - s"${tokens.mkString(params.delimiter.toString)}") - } else { - val indexSafeTokens = if (params.permissive && schema.length > tokens.length) { - tokens ++ new Array[String](schema.length - tokens.length) - } else if (params.permissive && schema.length < tokens.length) { - tokens.take(schema.length) - } else { - tokens - } - try { - var index: Int = 0 - var subIndex: Int = 0 - while (subIndex < safeRequiredIndices.length) { - index = safeRequiredIndices(subIndex) - // It anyway needs to try to parse since it decides if this row is malformed - // or not after trying to cast in `DROPMALFORMED` mode even if the casted - // value is not stored in the row. - val value = converters(index).apply(indexSafeTokens(index)) - if (subIndex < requiredSize) { - row(subIndex) = value - } - subIndex += 1 - } - Some(row) - } catch { - case NonFatal(e) if params.dropMalformed => - if (numMalformedRows < params.maxMalformedLogPerPartition) { - logWarning("Parse exception. " + - s"Dropping malformed line: ${tokens.mkString(params.delimiter.toString)}") - } - if (numMalformedRows == params.maxMalformedLogPerPartition - 1) { - logWarning( - s"More than ${params.maxMalformedLogPerPartition} malformed records have been " + - "found on this partition. Malformed records from now on will not be logged.") - } - None - } - } - } - } - // Skips the header line of each file if the `header` option is set to true. def dropHeaderLine( file: PartitionedFile, lines: Iterator[String], csvOptions: CSVOptions): Unit = { @@ -159,75 +67,3 @@ object CSVRelation extends Logging { } } } - -private[csv] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWriterFactory { - override def newInstance( - path: String, - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - new CsvOutputWriter(path, dataSchema, context, params) - } - - override def getFileExtension(context: TaskAttemptContext): String = { - ".csv" + CodecStreams.getCompressionExtension(context) - } -} - -private[csv] class CsvOutputWriter( - path: String, - dataSchema: StructType, - context: TaskAttemptContext, - params: CSVOptions) extends OutputWriter with Logging { - - // A `ValueConverter` is responsible for converting a value of an `InternalRow` to `String`. - // When the value is null, this converter should not be called. - private type ValueConverter = (InternalRow, Int) => String - - // `ValueConverter`s for all values in the fields of the schema - private val valueConverters: Array[ValueConverter] = - dataSchema.map(_.dataType).map(makeConverter).toArray - - private var printHeader: Boolean = params.headerFlag - private val writer = CodecStreams.createOutputStream(context, new Path(path)) - private val csvWriter = new LineCsvWriter(params, dataSchema.fieldNames.toSeq, writer) - - private def rowToString(row: InternalRow): Seq[String] = { - var i = 0 - val values = new Array[String](row.numFields) - while (i < row.numFields) { - if (!row.isNullAt(i)) { - values(i) = valueConverters(i).apply(row, i) - } else { - values(i) = params.nullValue - } - i += 1 - } - values - } - - private def makeConverter(dataType: DataType): ValueConverter = dataType match { - case DateType => - (row: InternalRow, ordinal: Int) => - params.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal))) - - case TimestampType => - (row: InternalRow, ordinal: Int) => - params.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal))) - - case udt: UserDefinedType[_] => makeConverter(udt.sqlType) - - case dt: DataType => - (row: InternalRow, ordinal: Int) => - row.get(ordinal, dt).toString - } - - override def write(row: InternalRow): Unit = { - csvWriter.writeRow(rowToString(row), printHeader) - printHeader = false - } - - override def close(): Unit = { - csvWriter.close() - writer.close() - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala new file mode 100644 index 0000000000000..ee79138c0f19b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.csv + +import java.io.Writer + +import com.univocity.parsers.csv.CsvWriter + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types._ + +private[csv] class UnivocityGenerator( + schema: StructType, + writer: Writer, + options: CSVOptions = new CSVOptions(Map.empty[String, String])) { + private val writerSettings = options.asWriterSettings + writerSettings.setHeaders(schema.fieldNames: _*) + private val gen = new CsvWriter(writer, writerSettings) + private var printHeader = options.headerFlag + + // A `ValueConverter` is responsible for converting a value of an `InternalRow` to `String`. + // When the value is null, this converter should not be called. + private type ValueConverter = (InternalRow, Int) => String + + // `ValueConverter`s for all values in the fields of the schema + private val valueConverters: Array[ValueConverter] = + schema.map(_.dataType).map(makeConverter).toArray + + private def makeConverter(dataType: DataType): ValueConverter = dataType match { + case DateType => + (row: InternalRow, ordinal: Int) => + options.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal))) + + case TimestampType => + (row: InternalRow, ordinal: Int) => + options.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal))) + + case udt: UserDefinedType[_] => makeConverter(udt.sqlType) + + case dt: DataType => + (row: InternalRow, ordinal: Int) => + row.get(ordinal, dt).toString + } + + private def convertRow(row: InternalRow): Seq[String] = { + var i = 0 + val values = new Array[String](row.numFields) + while (i < row.numFields) { + if (!row.isNullAt(i)) { + values(i) = valueConverters(i).apply(row, i) + } else { + values(i) = options.nullValue + } + i += 1 + } + values + } + + /** + * Writes a single InternalRow to CSV using Univocity. + */ + def write(row: InternalRow): Unit = { + if (printHeader) { + gen.writeHeaders() + } + gen.writeRow(convertRow(row): _*) + printHeader = false + } + + def close(): Unit = gen.close() + + def flush(): Unit = gen.flush() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala new file mode 100644 index 0000000000000..3b42aa60b024e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.csv + +import java.math.BigDecimal +import java.text.NumberFormat +import java.util.Locale + +import scala.util.Try +import scala.util.control.NonFatal + +import com.univocity.parsers.csv.CsvParser + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +private[csv] class UnivocityParser( + schema: StructType, + requiredSchema: StructType, + options: CSVOptions) extends Logging { + require(requiredSchema.toSet.subsetOf(schema.toSet), + "requiredSchema should be the subset of schema.") + + def this(schema: StructType, options: CSVOptions) = this(schema, schema, options) + + // A `ValueConverter` is responsible for converting the given value to a desired type. + private type ValueConverter = String => Any + + private val valueConverters = + schema.map(f => makeConverter(f.name, f.dataType, f.nullable, options)).toArray + + private val parser = new CsvParser(options.asParserSettings) + + private var numMalformedRecords = 0 + + private val row = new GenericInternalRow(requiredSchema.length) + + private val indexArr: Array[Int] = { + val fields = if (options.dropMalformed) { + // If `dropMalformed` is enabled, then it needs to parse all the values + // so that we can decide which row is malformed. + requiredSchema ++ schema.filterNot(requiredSchema.contains(_)) + } else { + requiredSchema + } + fields.map(schema.indexOf(_: StructField)).toArray + } + + /** + * Create a converter which converts the string value to a value according to a desired type. + * Currently, we do not support complex types (`ArrayType`, `MapType`, `StructType`). + * + * For other nullable types, returns null if it is null or equals to the value specified + * in `nullValue` option. + */ + def makeConverter( + name: String, + dataType: DataType, + nullable: Boolean = true, + options: CSVOptions = CSVOptions()): ValueConverter = dataType match { + case _: ByteType => (d: String) => + nullSafeDatum(d, name, nullable, options)(_.toByte) + + case _: ShortType => (d: String) => + nullSafeDatum(d, name, nullable, options)(_.toShort) + + case _: IntegerType => (d: String) => + nullSafeDatum(d, name, nullable, options)(_.toInt) + + case _: LongType => (d: String) => + nullSafeDatum(d, name, nullable, options)(_.toLong) + + case _: FloatType => (d: String) => + nullSafeDatum(d, name, nullable, options) { + case options.nanValue => Float.NaN + case options.negativeInf => Float.NegativeInfinity + case options.positiveInf => Float.PositiveInfinity + case datum => + Try(datum.toFloat) + .getOrElse(NumberFormat.getInstance(Locale.US).parse(datum).floatValue()) + } + + case _: DoubleType => (d: String) => + nullSafeDatum(d, name, nullable, options) { + case options.nanValue => Double.NaN + case options.negativeInf => Double.NegativeInfinity + case options.positiveInf => Double.PositiveInfinity + case datum => + Try(datum.toDouble) + .getOrElse(NumberFormat.getInstance(Locale.US).parse(datum).doubleValue()) + } + + case _: BooleanType => (d: String) => + nullSafeDatum(d, name, nullable, options)(_.toBoolean) + + case dt: DecimalType => (d: String) => + nullSafeDatum(d, name, nullable, options) { datum => + val value = new BigDecimal(datum.replaceAll(",", "")) + Decimal(value, dt.precision, dt.scale) + } + + case _: TimestampType => (d: String) => + nullSafeDatum(d, name, nullable, options) { datum => + // This one will lose microseconds parts. + // See https://issues.apache.org/jira/browse/SPARK-10681. + Try(options.timestampFormat.parse(datum).getTime * 1000L) + .getOrElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + DateTimeUtils.stringToTime(datum).getTime * 1000L + } + } + + case _: DateType => (d: String) => + nullSafeDatum(d, name, nullable, options) { datum => + // This one will lose microseconds parts. + // See https://issues.apache.org/jira/browse/SPARK-10681.x + Try(DateTimeUtils.millisToDays(options.dateFormat.parse(datum).getTime)) + .getOrElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime) + } + } + + case _: StringType => (d: String) => + nullSafeDatum(d, name, nullable, options)(UTF8String.fromString(_)) + + case udt: UserDefinedType[_] => (datum: String) => + makeConverter(name, udt.sqlType, nullable, options) + + case _ => throw new RuntimeException(s"Unsupported type: ${dataType.typeName}") + } + + private def nullSafeDatum( + datum: String, + name: String, + nullable: Boolean, + options: CSVOptions)(converter: ValueConverter): Any = { + if (datum == options.nullValue || datum == null) { + if (!nullable) { + throw new RuntimeException(s"null value found but field $name is not nullable.") + } + null + } else { + converter.apply(datum) + } + } + + /** + * Parses a single CSV record (in the form of an array of strings in which + * each element represents a column) and turns it into either one resulting row or no row (if the + * the record is malformed). + */ + def parse(input: String): Option[InternalRow] = { + convertWithParseMode(parser.parseLine(input)) { tokens => + var i: Int = 0 + while (i < indexArr.length) { + val pos = indexArr(i) + // It anyway needs to try to parse since it decides if this row is malformed + // or not after trying to cast in `DROPMALFORMED` mode even if the casted + // value is not stored in the row. + val value = valueConverters(pos).apply(tokens(pos)) + if (i < requiredSchema.length) { + row(i) = value + } + i += 1 + } + row + } + } + + private def convertWithParseMode( + tokens: Array[String])(convert: Array[String] => InternalRow): Option[InternalRow] = { + if (options.dropMalformed && schema.length != tokens.length) { + if (numMalformedRecords < options.maxMalformedLogPerPartition) { + logWarning(s"Dropping malformed line: ${tokens.mkString(options.delimiter.toString)}") + } + if (numMalformedRecords == options.maxMalformedLogPerPartition - 1) { + logWarning( + s"More than ${options.maxMalformedLogPerPartition} malformed records have been " + + "found on this partition. Malformed records from now on will not be logged.") + } + numMalformedRecords += 1 + None + } else if (options.failFast && schema.length != tokens.length) { + throw new RuntimeException(s"Malformed line in FAILFAST mode: " + + s"${tokens.mkString(options.delimiter.toString)}") + } else { + val checkedTokens = if (options.permissive && schema.length > tokens.length) { + tokens ++ new Array[String](schema.length - tokens.length) + } else if (options.permissive && schema.length < tokens.length) { + tokens.take(schema.length) + } else { + tokens + } + + try { + Some(convert(checkedTokens)) + } catch { + case NonFatal(e) if options.dropMalformed => + if (numMalformedRecords < options.maxMalformedLogPerPartition) { + logWarning("Parse exception. " + + s"Dropping malformed line: ${tokens.mkString(options.delimiter.toString)}") + } + if (numMalformedRecords == options.maxMalformedLogPerPartition - 1) { + logWarning( + s"More than ${options.maxMalformedLogPerPartition} malformed records have been " + + "found on this partition. Malformed records from now on will not be logged.") + } + numMalformedRecords += 1 + None + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 695ba1234d458..d10fa2c9ff247 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} -import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.command.{DDLUtils, RunnableCommand} import org.apache.spark.sql.types._ case class CreateTable( @@ -65,6 +65,11 @@ case class CreateTempViewUsing( } def run(sparkSession: SparkSession): Seq[Row] = { + if (provider.toLowerCase == DDLUtils.HIVE_PROVIDER) { + throw new AnalysisException("Hive data source can only be used with tables, " + + "you can't use it with CREATE TEMP VIEW USING") + } + val dataSource = DataSource( sparkSession, userSpecifiedSchema = userSpecifiedSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index b13849475811f..0590aec77c3cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -465,9 +465,9 @@ object JdbcUtils extends Logging { } (rs: ResultSet, row: InternalRow, pos: Int) => - val array = nullSafeConvert[Object]( - rs.getArray(pos + 1).getArray, - array => new GenericArrayData(elementConversion.apply(array))) + val array = nullSafeConvert[java.sql.Array]( + input = rs.getArray(pos + 1), + array => new GenericArrayData(elementConversion.apply(array.getArray))) row.update(pos, array) case _ => throw new IllegalArgumentException(s"Unsupported type ${dt.simpleString}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 5ca8226dbcdd6..d553d44cb3a14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources -import scala.util.control.NonFatal - import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ @@ -51,7 +49,7 @@ class ResolveDataSource(sparkSession: SparkSession) extends Rule[LogicalPlan] { // will catch it and return the original plan, so that the analyzer can report table not // found later. val isFileFormat = classOf[FileFormat].isAssignableFrom(dataSource.providingClass) - if (!isFileFormat) { + if (!isFileFormat || dataSource.className.toLowerCase == DDLUtils.HIVE_PROVIDER) { throw new AnalysisException("Unsupported data source type for direct query on files: " + s"${u.tableIdentifier.database.get}") } @@ -95,7 +93,7 @@ case class AnalyzeCreateTable(sparkSession: SparkSession) extends Rule[LogicalPl case c @ CreateTable(tableDesc, SaveMode.Append, Some(query)) if sparkSession.sessionState.catalog.tableExists(tableDesc.identifier) => // This is guaranteed by the parser and `DataFrameWriter` - assert(tableDesc.schema.isEmpty && tableDesc.provider.isDefined) + assert(tableDesc.provider.isDefined) // Analyze the query in CTAS and then we can do the normalization and checking. val qe = sparkSession.sessionState.executePlan(query) @@ -112,11 +110,6 @@ case class AnalyzeCreateTable(sparkSession: SparkSession) extends Rule[LogicalPl throw new AnalysisException("Saving data into a view is not allowed.") } - if (DDLUtils.isHiveTable(existingTable)) { - throw new AnalysisException(s"Saving data in the Hive serde table $tableName is " + - "not supported yet. Please use the insertInto() API as an alternative.") - } - // Check if the specified data source match the data source of the existing table. val existingProvider = DataSource.lookupDataSource(existingTable.provider.get) val specifiedProvider = DataSource.lookupDataSource(tableDesc.provider.get) @@ -186,9 +179,7 @@ case class AnalyzeCreateTable(sparkSession: SparkSession) extends Rule[LogicalPl } c.copy( - // trust everything from the existing table, except schema as we assume it's empty in a lot - // of places, when we do CTAS. - tableDesc = existingTable.copy(schema = new StructType()), + tableDesc = existingTable, query = Some(newQuery)) // Here we normalize partition, bucket and sort column names, w.r.t. the case sensitivity @@ -199,31 +190,55 @@ case class AnalyzeCreateTable(sparkSession: SparkSession) extends Rule[LogicalPl // * can't use all table columns as partition columns. // * partition columns' type must be AtomicType. // * sort columns' type must be orderable. + // * reorder table schema or output of query plan, to put partition columns at the end. case c @ CreateTable(tableDesc, _, query) => - val analyzedQuery = query.map { q => - // Analyze the query in CTAS and then we can do the normalization and checking. - val qe = sparkSession.sessionState.executePlan(q) + if (query.isDefined) { + assert(tableDesc.schema.isEmpty, + "Schema may not be specified in a Create Table As Select (CTAS) statement") + + val qe = sparkSession.sessionState.executePlan(query.get) qe.assertAnalyzed() - qe.analyzed - } - val schema = if (analyzedQuery.isDefined) { - analyzedQuery.get.schema - } else { - tableDesc.schema - } + val analyzedQuery = qe.analyzed + + val normalizedTable = normalizeCatalogTable(analyzedQuery.schema, tableDesc) + + val output = analyzedQuery.output + val partitionAttrs = normalizedTable.partitionColumnNames.map { partCol => + output.find(_.name == partCol).get + } + val newOutput = output.filterNot(partitionAttrs.contains) ++ partitionAttrs + val reorderedQuery = if (newOutput == output) { + analyzedQuery + } else { + Project(newOutput, analyzedQuery) + } - val columnNames = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { - schema.map(_.name) + c.copy(tableDesc = normalizedTable, query = Some(reorderedQuery)) } else { - schema.map(_.name.toLowerCase) + val normalizedTable = normalizeCatalogTable(tableDesc.schema, tableDesc) + + val partitionSchema = normalizedTable.partitionColumnNames.map { partCol => + normalizedTable.schema.find(_.name == partCol).get + } + + val reorderedSchema = + StructType(normalizedTable.schema.filterNot(partitionSchema.contains) ++ partitionSchema) + + c.copy(tableDesc = normalizedTable.copy(schema = reorderedSchema)) } - checkDuplication(columnNames, "table definition of " + tableDesc.identifier) + } - val normalizedTable = tableDesc.copy( - partitionColumnNames = normalizePartitionColumns(schema, tableDesc), - bucketSpec = normalizeBucketSpec(schema, tableDesc)) + private def normalizeCatalogTable(schema: StructType, table: CatalogTable): CatalogTable = { + val columnNames = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { + schema.map(_.name) + } else { + schema.map(_.name.toLowerCase) + } + checkDuplication(columnNames, "table definition of " + table.identifier) - c.copy(tableDesc = normalizedTable, query = analyzedQuery) + table.copy( + partitionColumnNames = normalizePartitionColumns(schema, table), + bucketSpec = normalizeBucketSpec(schema, table)) } private def normalizePartitionColumns(schema: StructType, table: CatalogTable): Seq[String] = { @@ -361,7 +376,7 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { } def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case i @ InsertIntoTable(table, partition, child, _, _) if table.resolved && child.resolved => + case i @ InsertIntoTable(table, _, child, _, _) if table.resolved && child.resolved => table match { case relation: CatalogRelation => val metadata = relation.catalogTable @@ -385,7 +400,8 @@ object HiveOnlyCheck extends (LogicalPlan => Unit) { plan.foreach { case CreateTable(tableDesc, _, Some(_)) if DDLUtils.isHiveTable(tableDesc) => throw new AnalysisException("Hive support is required to use CREATE Hive TABLE AS SELECT") - + case CreateTable(tableDesc, _, _) if DDLUtils.isHiveTable(tableDesc) => + throw new AnalysisException("Hive support is required to CREATE Hive TABLE") case _ => // OK } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index fde3b2a528994..2615eaa5ffe3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.plans.logical.LogicalState import org.apache.spark.sql.types.{DataType, ObjectType, StructType} @@ -144,6 +145,12 @@ object ObjectOperator { (i: InternalRow) => proj(i).get(0, deserializer.dataType) } + def deserializeRowToObject( + deserializer: Expression): InternalRow => Any = { + val proj = GenerateSafeProjection.generate(deserializer :: Nil) + (i: InternalRow) => proj(i).get(0, deserializer.dataType) + } + def serializeObjectToRow(serializer: Seq[Expression]): Any => UnsafeRow = { val proj = GenerateUnsafeProjection.generate(serializer) val objType = serializer.head.collect { case b: BoundReference => b.dataType }.head @@ -344,6 +351,21 @@ case class MapGroupsExec( } } +object MapGroupsExec { + def apply( + func: (Any, Iterator[Any], LogicalState[Any]) => TraversableOnce[Any], + keyDeserializer: Expression, + valueDeserializer: Expression, + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + outputObjAttr: Attribute, + child: SparkPlan): MapGroupsExec = { + val f = (key: Any, values: Iterator[Any]) => func(key, values, StateImpl[Any](None)) + new MapGroupsExec(f, keyDeserializer, valueDeserializer, + groupingAttributes, dataAttributes, outputObjAttr, child) + } +} + /** * Groups the input rows together and calls the R function with each group and an iterator * containing all elements in the group. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 6ab6fa61dc200..ef0cc74f3ffe3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.streaming +import java.util.concurrent.atomic.AtomicInteger + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{CurrentBatchTimestamp, Literal} import org.apache.spark.sql.SparkSession @@ -39,8 +41,9 @@ class IncrementalExecution( extends QueryExecution(sparkSession, logicalPlan) with Logging { // TODO: make this always part of planning. - val stateStrategy = + val streamingExtraStrategies = sparkSession.sessionState.planner.StatefulAggregationStrategy +: + sparkSession.sessionState.planner.MapGroupsWithStateStrategy +: sparkSession.sessionState.planner.StreamingRelationStrategy +: sparkSession.sessionState.experimentalMethods.extraStrategies @@ -49,7 +52,7 @@ class IncrementalExecution( new SparkPlanner( sparkSession.sparkContext, sparkSession.sessionState.conf, - stateStrategy) + streamingExtraStrategies) /** * See [SPARK-18339] @@ -58,7 +61,7 @@ class IncrementalExecution( */ override lazy val optimizedPlan: LogicalPlan = { sparkSession.sessionState.optimizer.execute(withCachedData) transformAllExpressions { - case ts @ CurrentBatchTimestamp(timestamp, _) => + case ts @ CurrentBatchTimestamp(timestamp, _, _) => logInfo(s"Current batch timestamp = $timestamp") ts.toLiteral } @@ -68,7 +71,7 @@ class IncrementalExecution( * Records the current id for a given stateful operator in the query plan as the `state` * preparation walks the query plan. */ - private var operatorId = 0 + private val operatorId = new AtomicInteger(0) /** Locates save/restore pairs surrounding aggregation. */ val state = new Rule[SparkPlan] { @@ -77,8 +80,8 @@ class IncrementalExecution( case StateStoreSaveExec(keys, None, None, None, UnaryExecNode(agg, StateStoreRestoreExec(keys2, None, child))) => - val stateId = OperatorStateId(checkpointLocation, operatorId, currentBatchId) - operatorId += 1 + val stateId = + OperatorStateId(checkpointLocation, operatorId.getAndIncrement(), currentBatchId) StateStoreSaveExec( keys, @@ -90,6 +93,14 @@ class IncrementalExecution( keys, Some(stateId), child) :: Nil)) + case MapGroupsWithStateExec( + func, keyDeser, valueDeser, groupAttr, dataAttr, outputAttr, + None, stateDeser, stateSer, child) => + val stateId = + OperatorStateId(checkpointLocation, operatorId.getAndIncrement(), currentBatchId) + MapGroupsWithStateExec( + func, keyDeser, valueDeser, groupAttr, dataAttr, outputAttr, + Some(stateId), stateDeser, stateSer, child) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index c5e9eae607b39..5eefca7aec955 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -239,7 +239,7 @@ trait ProgressReporter extends Logging { // Extract statistics about stateful operators in the query plan. val stateNodes = lastExecution.executedPlan.collect { - case p if p.isInstanceOf[StateStoreSaveExec] => p + case p if p.isInstanceOf[StateStoreWriter] => p } val stateOperators = stateNodes.map { node => new StateOperatorProgress( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala index 0551e4b4a2ef5..e6b4d1fa4d075 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala @@ -22,15 +22,17 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeProjection, Predicate} -import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark -import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalState} +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.execution +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.streaming.state._ -import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType +import org.apache.spark.util.CompletionIterator +import org.apache.spark.TaskContext /** Used to identify the state store for a given operator. */ @@ -53,6 +55,18 @@ trait StatefulOperator extends SparkPlan { } } +trait StateStoreReader extends StatefulOperator { + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) +} + +trait StateStoreWriter extends StatefulOperator { + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), + "numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows")) +} + /** * For each input tuple, the key is calculated and the value from the [[StateStore]] is added * to the stream (in addition to the input tuple) if present. @@ -63,9 +77,6 @@ case class StateStoreRestoreExec( child: SparkPlan) extends execution.UnaryExecNode with StatefulOperator { - override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) - override protected def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") @@ -101,12 +112,7 @@ case class StateStoreSaveExec( outputMode: Option[OutputMode] = None, eventTimeWatermark: Option[Long] = None, child: SparkPlan) - extends execution.UnaryExecNode with StatefulOperator { - - override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), - "numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows")) + extends execution.UnaryExecNode with StateStoreWriter { /** Generate a predicate that matches data older than the watermark */ private lazy val watermarkPredicate: Option[Predicate] = { @@ -150,6 +156,13 @@ case class StateStoreSaveExec( val numTotalStateRows = longMetric("numTotalStateRows") val numUpdatedStateRows = longMetric("numUpdatedStateRows") + // Abort the state store in case of error + TaskContext.get().addTaskCompletionListener(_ => { + if (!store.hasCommitted) { + store.abort() + } + }) + outputMode match { // Update and output all rows in the StateStore. case Some(Complete) => @@ -176,7 +189,7 @@ case class StateStoreSaveExec( } // Assumption: Append mode can be done only when watermark has been specified - store.remove(watermarkPredicate.get.eval) + store.remove(watermarkPredicate.get.eval _) store.commit() numTotalStateRows += store.numKeys() @@ -199,7 +212,7 @@ case class StateStoreSaveExec( override def hasNext: Boolean = { if (!baseIterator.hasNext) { // Remove old aggregates if watermark specified - if (watermarkPredicate.nonEmpty) store.remove(watermarkPredicate.get.eval) + if (watermarkPredicate.nonEmpty) store.remove(watermarkPredicate.get.eval _) store.commit() numTotalStateRows += store.numKeys() false @@ -227,3 +240,86 @@ case class StateStoreSaveExec( override def outputPartitioning: Partitioning = child.outputPartitioning } + + +/** + * Physical operator for executing streaming mapGroupsWithState. + */ +case class MapGroupsWithStateExec( + func: (Any, Iterator[Any], LogicalState[Any]) => Iterator[Any], + keyDeserializer: Expression, // probably not needed + valueDeserializer: Expression, + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + outputObjAttr: Attribute, + stateId: Option[OperatorStateId], + stateDeserializer: Expression, + stateSerializer: Seq[NamedExpression], + child: SparkPlan) extends UnaryExecNode with ObjectProducerExec with StateStoreWriter { + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(groupingAttributes) :: Nil + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + Seq(groupingAttributes.map(SortOrder(_, Ascending))) // is this ordering needed? + + override protected def doExecute(): RDD[InternalRow] = { + + child.execute().mapPartitionsWithStateStore[InternalRow]( + getStateId.checkpointLocation, + operatorId = getStateId.operatorId, + storeVersion = getStateId.batchId, + groupingAttributes.toStructType, + child.output.toStructType, + sqlContext.sessionState, + Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) => + try { + val numTotalStateRows = longMetric("numTotalStateRows") + val numUpdatedStateRows = longMetric("numUpdatedStateRows") + val numOutputRows = longMetric("numOutputRows") + + val groupedIter = GroupedIterator(iter, groupingAttributes, child.output) + + val getKeyObj = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) + val getKey = GenerateUnsafeProjection.generate(groupingAttributes, child.output) + val getValueObj = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) + val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val getStateObj = + ObjectOperator.deserializeRowToObject(stateDeserializer) + val outputStateObj = ObjectOperator.serializeObjectToRow(stateSerializer) + + val finalIterator = groupedIter.flatMap { case (keyRow, valueRowIter) => + val key = keyRow.asInstanceOf[UnsafeRow] + val keyObj = getKeyObj(keyRow) + val valueObjIter = valueRowIter.map(getValueObj.apply) + val stateObjOption = store.get(key).map(getStateObj) + val wrappedState = StateImpl[Any](stateObjOption) + val mappedIterator = func(keyObj, valueObjIter, wrappedState) + + if (wrappedState.isRemoved) { + store.remove(key) + numUpdatedStateRows += 1 + } else if (wrappedState.isUpdated) { + store.put(key, outputStateObj(wrappedState.get())) + numUpdatedStateRows += 1 + } + + mappedIterator.map { obj => + numOutputRows += 1 + getOutputRow(obj) + } + } + CompletionIterator[InternalRow, Iterator[InternalRow]](finalIterator, { + store.commit() + numTotalStateRows += store.numKeys() + }) + } catch { + case e: Throwable => + store.abort() + throw e + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index a35950e2dc17f..ea3719421b8a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -502,7 +502,7 @@ class StreamExecution( ct.dataType) case cd: CurrentDate => CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs, - cd.dataType) + cd.dataType, cd.timeZoneId) } reportTimeTaken("queryPlanning") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 4f3f8181d1f4e..26bde375f9739 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -147,6 +147,23 @@ private[state] class HDFSBackedStateStoreProvider( } } + override def remove(key: UnsafeRow): Unit = { + verify(state == UPDATING, "Cannot remove after already committed or aborted") + if (mapToUpdate.containsKey(key)) { + val value = mapToUpdate.remove(key) + Option(allUpdates.get(key)) match { + case Some(ValueUpdated(_, _)) | None => + // Value existed in previous version and maybe was updated, mark removed + allUpdates.put(key, ValueRemoved(key, value)) + case Some(ValueAdded(_, _)) => + // Value did not exist in previous version and was added, should not appear in updates + allUpdates.remove(key) + case Some(ValueRemoved(_, _)) => + // Remove already in update map, no need to change + } + } + } + /** Commit all the updates that have been made to the store, and return the new version. */ override def commit(): Long = { verify(state == UPDATING, "Cannot commit after already committed or aborted") @@ -203,7 +220,7 @@ private[state] class HDFSBackedStateStoreProvider( /** * Whether all updates have been committed */ - override private[state] def hasCommitted: Boolean = { + override private[streaming] def hasCommitted: Boolean = { state == COMMITTED } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 9bc6c0e2b9334..dc2bcee95ca65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.streaming.state import java.util.concurrent.{ScheduledFuture, TimeUnit} +import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import scala.util.control.NonFatal @@ -57,6 +58,8 @@ trait StateStore { */ def remove(condition: UnsafeRow => Boolean): Unit + def remove(key: UnsafeRow): Unit + /** * Commit all the updates that have been made to the store, and return the new version. */ @@ -83,7 +86,7 @@ trait StateStore { /** * Whether all updates have been committed */ - private[state] def hasCommitted: Boolean + private[streaming] def hasCommitted: Boolean } @@ -124,12 +127,46 @@ object StateStore extends Logging { val MAINTENANCE_INTERVAL_CONFIG = "spark.sql.streaming.stateStore.maintenanceInterval" val MAINTENANCE_INTERVAL_DEFAULT_SECS = 60 + @GuardedBy("loadedProviders") private val loadedProviders = new mutable.HashMap[StateStoreId, StateStoreProvider]() - private val maintenanceTaskExecutor = - ThreadUtils.newDaemonSingleThreadScheduledExecutor("state-store-maintenance-task") - @volatile private var maintenanceTask: ScheduledFuture[_] = null - @volatile private var _coordRef: StateStoreCoordinatorRef = null + /** + * Runs the `task` periodically and automatically cancels it if there is an exception. `onError` + * will be called when an exception happens. + */ + class MaintenanceTask(periodMs: Long, task: => Unit, onError: => Unit) { + private val executor = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("state-store-maintenance-task") + + private val runnable = new Runnable { + override def run(): Unit = { + try { + task + } catch { + case NonFatal(e) => + logWarning("Error running maintenance thread", e) + onError + throw e + } + } + } + + private val future: ScheduledFuture[_] = executor.scheduleAtFixedRate( + runnable, periodMs, periodMs, TimeUnit.MILLISECONDS) + + def stop(): Unit = { + future.cancel(false) + executor.shutdown() + } + + def isRunning: Boolean = !future.isDone + } + + @GuardedBy("loadedProviders") + private var maintenanceTask: MaintenanceTask = null + + @GuardedBy("loadedProviders") + private var _coordRef: StateStoreCoordinatorRef = null /** Get or create a store associated with the id. */ def get( @@ -162,7 +199,7 @@ object StateStore extends Logging { } def isMaintenanceRunning: Boolean = loadedProviders.synchronized { - maintenanceTask != null + maintenanceTask != null && maintenanceTask.isRunning } /** Unload and stop all state store providers */ @@ -170,7 +207,7 @@ object StateStore extends Logging { loadedProviders.clear() _coordRef = null if (maintenanceTask != null) { - maintenanceTask.cancel(false) + maintenanceTask.stop() maintenanceTask = null } logInfo("StateStore stopped") @@ -179,14 +216,14 @@ object StateStore extends Logging { /** Start the periodic maintenance task if not already started and if Spark active */ private def startMaintenanceIfNeeded(): Unit = loadedProviders.synchronized { val env = SparkEnv.get - if (maintenanceTask == null && env != null) { + if (env != null && !isMaintenanceRunning) { val periodMs = env.conf.getTimeAsMs( MAINTENANCE_INTERVAL_CONFIG, s"${MAINTENANCE_INTERVAL_DEFAULT_SECS}s") - val runnable = new Runnable { - override def run(): Unit = { doMaintenance() } - } - maintenanceTask = maintenanceTaskExecutor.scheduleAtFixedRate( - runnable, periodMs, periodMs, TimeUnit.MILLISECONDS) + maintenanceTask = new MaintenanceTask( + periodMs, + task = { doMaintenance() }, + onError = { loadedProviders.synchronized { loadedProviders.clear() } } + ) logInfo("State Store maintenance task started") } } @@ -198,21 +235,20 @@ object StateStore extends Logging { private def doMaintenance(): Unit = { logDebug("Doing maintenance") if (SparkEnv.get == null) { - stop() - } else { - loadedProviders.synchronized { loadedProviders.toSeq }.foreach { case (id, provider) => - try { - if (verifyIfStoreInstanceActive(id)) { - provider.doMaintenance() - } else { - unload(id) - logInfo(s"Unloaded $provider") - } - } catch { - case NonFatal(e) => - logWarning(s"Error managing $provider, stopping management thread") - stop() + throw new IllegalStateException("SparkEnv not active, cannot do maintenance on StateStores") + } + loadedProviders.synchronized { loadedProviders.toSeq }.foreach { case (id, provider) => + try { + if (verifyIfStoreInstanceActive(id)) { + provider.doMaintenance() + } else { + unload(id) + logInfo(s"Unloaded $provider") } + } catch { + case NonFatal(e) => + logWarning(s"Error managing $provider, stopping management thread") + throw e } } } @@ -238,7 +274,7 @@ object StateStore extends Logging { } } - private def coordinatorRef: Option[StateStoreCoordinatorRef] = synchronized { + private def coordinatorRef: Option[StateStoreCoordinatorRef] = loadedProviders.synchronized { val env = SparkEnv.get if (env != null) { if (_coordRef == null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 9a080fd3c97c1..c86ae5be9ef62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -633,7 +633,7 @@ object functions { def skewness(columnName: String): Column = skewness(Column(columnName)) /** - * Aggregate function: alias for [[stddev_samp]]. + * Aggregate function: alias for `stddev_samp`. * * @group agg_funcs * @since 1.6.0 @@ -641,7 +641,7 @@ object functions { def stddev(e: Column): Column = withAggregateFunction { StddevSamp(e.expr) } /** - * Aggregate function: alias for [[stddev_samp]]. + * Aggregate function: alias for `stddev_samp`. * * @group agg_funcs * @since 1.6.0 @@ -717,7 +717,7 @@ object functions { def sumDistinct(columnName: String): Column = sumDistinct(Column(columnName)) /** - * Aggregate function: alias for [[var_samp]]. + * Aggregate function: alias for `var_samp`. * * @group agg_funcs * @since 1.6.0 @@ -725,7 +725,7 @@ object functions { def variance(e: Column): Column = withAggregateFunction { VarianceSamp(e.expr) } /** - * Aggregate function: alias for [[var_samp]]. + * Aggregate function: alias for `var_samp`. * * @group agg_funcs * @since 1.6.0 @@ -2870,6 +2870,15 @@ object functions { */ def explode(e: Column): Column = withExpr { Explode(e.expr) } + /** + * Creates a new row for each element in the given array or map column. + * Unlike explode, if the array/map is null or empty then null is produced. + * + * @group collection_funcs + * @since 2.2.0 + */ + def explode_outer(e: Column): Column = withExpr { GeneratorOuter(Explode(e.expr)) } + /** * Creates a new row for each element with position in the given array or map column. * @@ -2878,6 +2887,15 @@ object functions { */ def posexplode(e: Column): Column = withExpr { PosExplode(e.expr) } + /** + * Creates a new row for each element with position in the given array or map column. + * Unlike posexplode, if the array/map is null or empty then the row (null, null) is produced. + * + * @group collection_funcs + * @since 2.2.0 + */ + def posexplode_outer(e: Column): Column = withExpr { GeneratorOuter(PosExplode(e.expr)) } + /** * Extracts json object from a json string based on json path specified, and returns json string * of the extracted json object. It will return null if the input json string is invalid. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 8244b2152c1c4..3d9f41832bc73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.internal -import scala.collection.JavaConverters._ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.Experimental @@ -257,101 +256,74 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { /** * :: Experimental :: - * Creates an external table from the given path and returns the corresponding DataFrame. + * Creates a table from the given path and returns the corresponding DataFrame. * It will use the default data source configured by spark.sql.sources.default. * * @group ddl_ops - * @since 2.0.0 + * @since 2.2.0 */ @Experimental - override def createExternalTable(tableName: String, path: String): DataFrame = { + override def createTable(tableName: String, path: String): DataFrame = { val dataSourceName = sparkSession.sessionState.conf.defaultDataSourceName - createExternalTable(tableName, path, dataSourceName) + createTable(tableName, path, dataSourceName) } /** * :: Experimental :: - * Creates an external table from the given path based on a data source - * and returns the corresponding DataFrame. + * Creates a table from the given path based on a data source and returns the corresponding + * DataFrame. * * @group ddl_ops - * @since 2.0.0 + * @since 2.2.0 */ @Experimental - override def createExternalTable(tableName: String, path: String, source: String): DataFrame = { - createExternalTable(tableName, source, Map("path" -> path)) - } - - /** - * :: Experimental :: - * Creates an external table from the given path based on a data source and a set of options. - * Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 2.0.0 - */ - @Experimental - override def createExternalTable( - tableName: String, - source: String, - options: java.util.Map[String, String]): DataFrame = { - createExternalTable(tableName, source, options.asScala.toMap) + override def createTable(tableName: String, path: String, source: String): DataFrame = { + createTable(tableName, source, Map("path" -> path)) } /** * :: Experimental :: * (Scala-specific) - * Creates an external table from the given path based on a data source and a set of options. + * Creates a table from the given path based on a data source and a set of options. * Then, returns the corresponding DataFrame. * * @group ddl_ops - * @since 2.0.0 + * @since 2.2.0 */ @Experimental - override def createExternalTable( + override def createTable( tableName: String, source: String, options: Map[String, String]): DataFrame = { - createExternalTable(tableName, source, new StructType, options) - } - - /** - * :: Experimental :: - * Create an external table from the given path based on a data source, a schema and - * a set of options. Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 2.0.0 - */ - @Experimental - override def createExternalTable( - tableName: String, - source: String, - schema: StructType, - options: java.util.Map[String, String]): DataFrame = { - createExternalTable(tableName, source, schema, options.asScala.toMap) + createTable(tableName, source, new StructType, options) } /** * :: Experimental :: * (Scala-specific) - * Create an external table from the given path based on a data source, a schema and - * a set of options. Then, returns the corresponding DataFrame. + * Create a table from the given path based on a data source, a schema and a set of options. + * Then, returns the corresponding DataFrame. * * @group ddl_ops - * @since 2.0.0 + * @since 2.2.0 */ @Experimental - override def createExternalTable( + override def createTable( tableName: String, source: String, schema: StructType, options: Map[String, String]): DataFrame = { val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) + val storage = DataSource.buildStorageFormatFromOptions(options) + val tableType = if (storage.locationUri.isDefined) { + CatalogTableType.EXTERNAL + } else { + CatalogTableType.MANAGED + } val tableDesc = CatalogTable( identifier = tableIdent, - tableType = CatalogTableType.EXTERNAL, - storage = DataSource.buildStorageFormatFromOptions(options), + tableType = tableType, + storage = storage, schema = schema, provider = Some(source) ) @@ -468,7 +440,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { // If this table is cached as an InMemoryRelation, drop the original // cached version and make the new version cached lazily. - val logicalPlan = sparkSession.sessionState.catalog.lookupRelation(tableIdent) + val logicalPlan = sparkSession.table(tableIdent).queryExecution.analyzed // Use lookupCachedData directly since RefreshTable also takes databaseName. val isCached = sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty if (isCached) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 645b0fa13ee3f..5ba4192512a59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.internal -import java.util.{NoSuchElementException, Properties} +import java.util.{NoSuchElementException, Properties, TimeZone} import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ @@ -50,6 +50,11 @@ object SQLConf { sqlConfEntries.put(entry.key, entry) } + // For testing only + private[sql] def unregister(entry: ConfigEntry[_]): Unit = sqlConfEntries.synchronized { + sqlConfEntries.remove(entry.key) + } + private[sql] object SQLConfigBuilder { def apply(key: String): ConfigBuilder = new ConfigBuilder(key).onCreate(register) @@ -655,6 +660,12 @@ object SQLConf { .booleanConf .createWithDefault(false) + val SESSION_LOCAL_TIMEZONE = + SQLConfigBuilder("spark.sql.session.timeZone") + .doc("""The ID of session local timezone, e.g. "GMT", "America/Los_Angeles", etc.""") + .stringConf + .createWithDefault(TimeZone.getDefault().getID()) + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -853,6 +864,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { override def crossJoinEnabled: Boolean = getConf(SQLConf.CROSS_JOINS_ENABLED) + override def sessionLocalTimeZone: String = getConf(SQLConf.SESSION_LOCAL_TIMEZONE) + def ndvMaxError: Double = getConf(NDV_MAX_ERROR) override def cboEnabled: Boolean = getConf(SQLConf.CBO_ENABLED) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 64ec62f41d1f8..68b774b52fd7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -114,12 +114,14 @@ private[sql] class SessionState(sparkSession: SparkSession) { lazy val analyzer: Analyzer = { new Analyzer(catalog, conf) { override val extendedResolutionRules = - AnalyzeCreateTable(sparkSession) :: - PreprocessTableInsertion(conf) :: new FindDataSourceTable(sparkSession) :: - DataSourceAnalysis(conf) :: new ResolveDataSource(sparkSession) :: Nil + override val postHocResolutionRules = + AnalyzeCreateTable(sparkSession) :: + PreprocessTableInsertion(conf) :: + DataSourceAnalysis(conf) :: Nil + override val extendedCheckRules = Seq(PreWriteCheck(conf, catalog), HiveOnlyCheck) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 7db9d9264b1c9..6d2cede6f5a23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -22,6 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.StreamingRelation import org.apache.spark.sql.types.StructType @@ -116,6 +117,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * @since 2.0.0 */ def load(): DataFrame = { + if (source.toLowerCase == DDLUtils.HIVE_PROVIDER) { + throw new AnalysisException("Hive data source can only be used with tables, you can not " + + "read files of Hive data source directly.") + } + val dataSource = DataSource( sparkSession, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 7e7a1ba22334e..0f7a33723cccc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -22,6 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, ForeachWriter} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ +import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.{ForeachSink, MemoryPlan, MemorySink} @@ -221,6 +222,11 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * @since 2.0.0 */ def start(): StreamingQuery = { + if (source.toLowerCase == DDLUtils.HIVE_PROVIDER) { + throw new AnalysisException("Hive data source can only be used with tables, you can not " + + "write files of Hive data source directly.") + } + if (source == "memory") { assertNotPartitioned("memory") if (extraOptions.get("queryName").isEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 7b9770dadd0f6..0b9406b027f53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -230,6 +230,12 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) { UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode) } + if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { + throw new AnalysisException( + s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} " + + "is not supported in streaming DataFrames/Datasets") + } + new StreamingQueryWrapper(new StreamExecution( sparkSession, userSpecifiedName.orNull, diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 8304b728aa238..6260b6127d52e 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -225,6 +225,38 @@ public Iterator call(Integer key, Iterator values) { Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped.collectAsList())); + Dataset mapped2 = grouped.mapGroupsWithState( + new MapGroupsWithStateFunction() { + @Override + public String call(Integer key, Iterator values, State s) throws Exception { + StringBuilder sb = new StringBuilder(key.toString()); + while (values.hasNext()) { + sb.append(values.next()); + } + return sb.toString(); + } + }, + Encoders.LONG(), + Encoders.STRING()); + + Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped2.collectAsList())); + + Dataset flatMapped2 = grouped.flatMapGroupsWithState( + new FlatMapGroupsWithStateFunction() { + @Override + public Iterator call(Integer key, Iterator values, State s) { + StringBuilder sb = new StringBuilder(key.toString()); + while (values.hasNext()) { + sb.append(values.next()); + } + return Collections.singletonList(sb.toString()).iterator(); + } + }, + Encoders.LONG(), + Encoders.STRING()); + + Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped2.collectAsList())); + Dataset> reduced = grouped.reduceGroups(new ReduceFunction() { @Override public String call(String v1, String v2) throws Exception { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index 8bf3278c43880..bbaac5a33975b 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -18,6 +18,7 @@ package test.org.apache.spark.sql; import java.io.Serializable; +import java.util.List; import org.junit.After; import org.junit.Assert; @@ -108,4 +109,25 @@ public void udf3Test() { result = spark.sql("SELECT stringLengthTest('test', 'test2')").head(); Assert.assertEquals(9, result.getInt(0)); } + + @SuppressWarnings("unchecked") + @Test + public void udf4Test() { + spark.udf().register("inc", new UDF1() { + @Override + public Long call(Long i) { + return i + 1; + } + }, DataTypes.LongType); + + spark.range(10).toDF("x").createOrReplaceTempView("tmp"); + // This tests when Java UDFs are required to be the semantically same (See SPARK-9435). + List results = spark.sql("SELECT inc(x) FROM tmp GROUP BY inc(x)").collectAsList(); + Assert.assertEquals(10, results.size()); + long sum = 0; + for (Row result : results) { + sum += result.getLong(0); + } + Assert.assertEquals(55, sum); + } } diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index 25b817382195a..2e5cac12952db 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -53,5 +53,5 @@ log4j.additivity.hive.ql.metadata.Hive=false log4j.logger.hive.ql.metadata.Hive=OFF # Parquet related logging -log4j.logger.org.apache.parquet=ERROR -log4j.logger.parquet=ERROR +log4j.logger.org.apache.parquet.CorruptStatistics=ERROR +log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 818b19c50f16f..ad0f885f63d3d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -1,5 +1,5 @@ -- Create the origin table -CREATE TABLE test_change(a INT, b STRING, c INT); +CREATE TABLE test_change(a INT, b STRING, c INT) using parquet; DESC test_change; -- Change column name (not supported yet) @@ -47,7 +47,7 @@ CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one"; ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column a'; -- Change column in partition spec (not supported yet) -CREATE TABLE partition_table(a INT, b STRING) PARTITIONED BY (c INT, d STRING); +CREATE TABLE partition_table(a INT, b STRING, c INT, d STRING) USING parquet PARTITIONED BY (c, d); ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT; -- DROP TEST TABLE diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe.sql b/sql/core/src/test/resources/sql-tests/inputs/describe.sql index 84503d0b12a8e..ff327f5e82b13 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/describe.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/describe.sql @@ -1,4 +1,4 @@ -CREATE TABLE t (a STRING, b INT) PARTITIONED BY (c STRING, d STRING); +CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d); ALTER TABLE t ADD PARTITION (c='Us', d=1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql b/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql index 18d02e150ec67..10c379dfa014e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql @@ -1,9 +1,9 @@ -- Test data. CREATE DATABASE showdb; USE showdb; -CREATE TABLE show_t1(a String, b Int) PARTITIONED BY (c String, d String); +CREATE TABLE show_t1(a String, b Int, c String, d String) USING parquet PARTITIONED BY (c, d); ALTER TABLE show_t1 ADD PARTITION (c='Us', d=1); -CREATE TABLE show_t2(b String, d Int); +CREATE TABLE show_t2(b String, d Int) USING parquet; CREATE TEMPORARY VIEW show_t3(e int) USING parquet; CREATE GLOBAL TEMP VIEW show_t4 AS SELECT 1 as col1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql b/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql index 3894082255088..1e02c2f045ea9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql @@ -2,8 +2,8 @@ CREATE DATABASE showdb; USE showdb; -CREATE TABLE showcolumn1 (col1 int, `col 2` int); -CREATE TABLE showcolumn2 (price int, qty int) partitioned by (year int, month int); +CREATE TABLE showcolumn1 (col1 int, `col 2` int) USING parquet; +CREATE TABLE showcolumn2 (price int, qty int, year int, month int) USING parquet partitioned by (year, month); CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING parquet; CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5`; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-basic.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-basic.sql new file mode 100644 index 0000000000000..332e858800f7c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-basic.sql @@ -0,0 +1,123 @@ +-- Tests EXISTS subquery support. Tests basic form +-- of EXISTS subquery (both EXISTS and NOT EXISTS) + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- uncorrelated exist query +-- TC.01.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30); + +-- simple correlated predicate in exist subquery +-- TC.01.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id); + +-- correlated outer isnull predicate +-- TC.01.03 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id + OR emp.dept_id IS NULL); + +-- Simple correlation with a local predicate in outer query +-- TC.01.04 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200; + +-- Outer references (emp.id) should not be pruned from outer plan +-- TC.01.05 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200; + +-- not exists with correlated predicate +-- TC.01.06 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id); + +-- not exists with correlated predicate + local predicate +-- TC.01.07 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id + OR state = 'NJ'); + +-- not exist both equal and greaterthan predicate +-- TC.01.08 +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary); + +-- select employees who have not received any bonus +-- TC 01.09 +SELECT emp.* +FROM emp +WHERE NOT EXISTS (SELECT NULL + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + +-- Nested exists +-- TC.01.10 +SELECT * +FROM bonus +WHERE EXISTS (SELECT emp_name + FROM emp + WHERE bonus.emp_name = emp.emp_name + AND EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-within-and-or.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-within-and-or.sql new file mode 100644 index 0000000000000..7743b5241d11d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-within-and-or.sql @@ -0,0 +1,96 @@ +-- Tests EXISTS subquery support. Tests EXISTS +-- subquery within a AND or OR expression. + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + + +-- Or used in conjunction with exists - ExistenceJoin +-- TC.02.01 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.id > 200; + +-- all records from emp including the null dept_id +-- TC.02.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.dept_id IS NULL; + +-- EXISTS subquery in both LHS and RHS of OR. +-- TC.02.03 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 20) + OR EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 30); +; + +-- not exists and exists predicate within OR +-- TC.02.04 +SELECT * +FROM bonus +WHERE ( NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) + OR EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + OR bonus_amt < emp.salary) ); + +-- not exists and in predicate within AND +-- TC.02.05 +SELECT * FROM bonus WHERE NOT EXISTS +( + SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +AND +emp_name IN +( + SELECT emp_name + FROM emp + WHERE bonus_amt < emp.salary); + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-multiple-columns.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-multiple-columns.sql new file mode 100644 index 0000000000000..db668505adf24 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-multiple-columns.sql @@ -0,0 +1,55 @@ +-- This file contains test cases for NOT IN subquery with multiple columns. + +-- The data sets are populated as follows: +-- 1) When T1.A1 = T2.A2 +-- 1.1) T1.B1 = T2.B2 +-- 1.2) T1.B1 = T2.B2 returns false +-- 1.3) T1.B1 is null +-- 1.4) T2.B2 is null +-- 2) When T1.A1 = T2.A2 returns false +-- 3) When T1.A1 is null +-- 4) When T1.A2 is null + +-- T1.A1 T1.B1 T2.A2 T2.B2 +-- ----- ----- ----- ----- +-- 1 1 1 1 (1.1) +-- 1 3 (1.2) +-- 1 null 1 null (1.3 & 1.4) +-- +-- 2 1 1 1 (2) +-- null 1 (3) +-- null 3 (4) + +create temporary view t1 as select * from values + (1, 1), (2, 1), (null, 1), + (1, 3), (null, 3), + (1, null), (null, 2) +as t1(a1, b1); + +create temporary view t2 as select * from values + (1, 1), + (null, 3), + (1, null) +as t2(a2, b2); + +-- multiple columns in NOT IN +-- TC 01.01 +select a1,b1 +from t1 +where (a1,b1) not in (select a2,b2 + from t2); + +-- multiple columns with expressions in NOT IN +-- TC 01.02 +select a1,b1 +from t1 +where (a1-1,b1) not in (select a2,b2 + from t2); + +-- multiple columns with expressions in NOT IN +-- TC 01.02 +select a1,b1 +from t1 +where (a1,b1) not in (select a2+1,b2 + from t2); + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql new file mode 100644 index 0000000000000..cf93c5a835971 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql @@ -0,0 +1,42 @@ +-- The test file contains negative test cases +-- of invalid queries where error messages are expected. + +create temporary view t1 as select * from values + (1, 2, 3) +as t1(t1a, t1b, t1c); + +create temporary view t2 as select * from values + (1, 0, 1) +as t2(t2a, t2b, t2c); + +create temporary view t3 as select * from values + (3, 1, 2) +as t3(t3a, t3b, t3c); + +-- TC 01.01 +-- The column t2b in the SELECT of the subquery is invalid +-- because it is neither an aggregate function nor a GROUP BY column. +select t1a, t2b +from t1, t2 +where t1b = t2c +and t2b = (select max(avg) + from (select t2b, avg(t2b) avg + from t2 + where t2a = t1.t1b + ) + ) +; + +-- TC 01.02 +-- Invalid due to the column t2b not part of the output from table t2. +select * +from t1 +where t1a in (select min(t2a) + from t2 + group by t2c + having t2c in (select max(t3c) + from t3 + group by t3b + having t3b > t2b )) +; + diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 156ddb86adf9c..59eb56920cdcf 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -3,7 +3,7 @@ -- !query 0 -CREATE TABLE test_change(a INT, b STRING, c INT) +CREATE TABLE test_change(a INT, b STRING, c INT) using parquet -- !query 0 schema struct<> -- !query 0 output @@ -269,7 +269,7 @@ Database 'global_temp' not found; -- !query 28 -CREATE TABLE partition_table(a INT, b STRING) PARTITIONED BY (c INT, d STRING) +CREATE TABLE partition_table(a INT, b STRING, c INT, d STRING) USING parquet PARTITIONED BY (c, d) -- !query 28 schema struct<> -- !query 28 output diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index b448d60c7685d..0a11c1cde2b45 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -3,7 +3,7 @@ -- !query 0 -CREATE TABLE t (a STRING, b INT) PARTITIONED BY (c STRING, d STRING) +CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d) -- !query 0 schema struct<> -- !query 0 output diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index 904601bf1133c..3d287f43accc9 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -19,7 +19,7 @@ struct<> -- !query 2 -CREATE TABLE show_t1(a String, b Int) PARTITIONED BY (c String, d String) +CREATE TABLE show_t1(a String, b Int, c String, d String) USING parquet PARTITIONED BY (c, d) -- !query 2 schema struct<> -- !query 2 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -CREATE TABLE show_t2(b String, d Int) +CREATE TABLE show_t2(b String, d Int) USING parquet -- !query 4 schema struct<> -- !query 4 output diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out index 832e6e25bb2bd..05c3a083ee3b3 100644 --- a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out @@ -19,7 +19,7 @@ struct<> -- !query 2 -CREATE TABLE showcolumn1 (col1 int, `col 2` int) +CREATE TABLE showcolumn1 (col1 int, `col 2` int) USING parquet -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -CREATE TABLE showcolumn2 (price int, qty int) partitioned by (year int, month int) +CREATE TABLE showcolumn2 (price int, qty int, year int, month int) USING parquet partitioned by (year, month) -- !query 3 schema struct<> -- !query 3 output diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out new file mode 100644 index 0000000000000..900e4d573bef1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out @@ -0,0 +1,214 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query 0 +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * +FROM emp +WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) +-- !query 3 schema +struct +-- !query 3 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 4 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) +-- !query 4 schema +struct +-- !query 4 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 5 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id + OR emp.dept_id IS NULL) +-- !query 5 schema +struct +-- !query 5 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +800 emp 8 2016-01-01 150.0 70 + + +-- !query 6 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200 +-- !query 6 schema +struct +-- !query 6 output +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 7 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200 +-- !query 7 schema +struct +-- !query 7 output +emp 3 +emp 4 +emp 8 + + +-- !query 8 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id) +-- !query 8 schema +struct +-- !query 8 output +40 dept 4 - unassigned OR +50 dept 5 - unassigned NJ + + +-- !query 9 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id + OR state = 'NJ') +-- !query 9 schema +struct +-- !query 9 output +40 dept 4 - unassigned OR + + +-- !query 10 +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +-- !query 10 schema +struct +-- !query 10 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 4 100.0 + + +-- !query 11 +SELECT emp.* +FROM emp +WHERE NOT EXISTS (SELECT NULL + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query 11 schema +struct +-- !query 11 output +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 12 +SELECT * +FROM bonus +WHERE EXISTS (SELECT emp_name + FROM emp + WHERE bonus.emp_name = emp.emp_name + AND EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +-- !query 12 schema +struct +-- !query 12 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out new file mode 100644 index 0000000000000..865e4ed14e4ab --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out @@ -0,0 +1,156 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.id > 200 +-- !query 3 schema +struct +-- !query 3 output +emp 1 +emp 1 +emp 2 +emp 3 +emp 4 +emp 5 +emp 6 - no dept +emp 7 +emp 8 + + +-- !query 4 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.dept_id IS NULL +-- !query 4 schema +struct +-- !query 4 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +800 emp 8 2016-01-01 150.0 70 + + +-- !query 5 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 20) + OR EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 30) +-- !query 5 schema +struct +-- !query 5 output +emp 3 +emp 4 + + +-- !query 6 +SELECT * +FROM bonus +WHERE ( NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) + OR EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + OR bonus_amt < emp.salary) ) +-- !query 6 schema +struct +-- !query 6 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 + + +-- !query 7 +SELECT * FROM bonus WHERE NOT EXISTS +( + SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +AND +emp_name IN +( + SELECT emp_name + FROM emp + WHERE bonus_amt < emp.salary) +-- !query 7 schema +struct +-- !query 7 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 4 100.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-multiple-columns.sql.out new file mode 100644 index 0000000000000..756c3782a0e7a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-multiple-columns.sql.out @@ -0,0 +1,59 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +create temporary view t1 as select * from values + (1, 1), (2, 1), (null, 1), + (1, 3), (null, 3), + (1, null), (null, 2) +as t1(a1, b1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + (1, 1), + (null, 3), + (1, null) +as t2(a2, b2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +select a1,b1 +from t1 +where (a1,b1) not in (select a2,b2 + from t2) +-- !query 2 schema +struct +-- !query 2 output +2 1 + + +-- !query 3 +select a1,b1 +from t1 +where (a1-1,b1) not in (select a2,b2 + from t2) +-- !query 3 schema +struct +-- !query 3 output +1 1 + + +-- !query 4 +select a1,b1 +from t1 +where (a1,b1) not in (select a2+1,b2 + from t2) +-- !query 4 schema +struct +-- !query 4 output +1 1 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out new file mode 100644 index 0000000000000..50ae01e181bcf --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -0,0 +1,66 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +create temporary view t1 as select * from values + (1, 2, 3) +as t1(t1a, t1b, t1c) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + (1, 0, 1) +as t2(t2a, t2b, t2c) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + (3, 1, 2) +as t3(t3a, t3b, t3c) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +select t1a, t2b +from t1, t2 +where t1b = t2c +and t2b = (select max(avg) + from (select t2b, avg(t2b) avg + from t2 + where t2a = t1.t1b + ) + ) +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException +expression 't2.`t2b`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; + + +-- !query 4 +select * +from t1 +where t1a in (select min(t2a) + from t2 + group by t2c + having t2c in (select max(t3c) + from t3 + group by t3b + having t3b > t2b )) +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +resolved attribute(s) t2b#x missing from min(t2a)#x,t2c#x in operator !Filter predicate-subquery#x [(t2c#x = max(t3c)#x) && (t3b#x > t2b#x)]; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 339262a5a34e5..1af1a3652971c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -39,6 +39,8 @@ private case class BigData(s: String) class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext { import testImplicits._ + setupTestData() + override def afterEach(): Unit = { try { spark.catalog.clearCache() @@ -185,9 +187,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext assertCached(spark.table("testData")) assertResult(1, "InMemoryRelation not found, testData should have been cached") { - spark.table("testData").queryExecution.withCachedData.collect { - case r: InMemoryRelation => r - }.size + getNumInMemoryRelations(spark.table("testData").queryExecution.withCachedData) } spark.catalog.cacheTable("testData") @@ -580,10 +580,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext localRelation.createOrReplaceTempView("localRelation") spark.catalog.cacheTable("localRelation") - assert( - localRelation.queryExecution.withCachedData.collect { - case i: InMemoryRelation => i - }.size == 1) + assert(getNumInMemoryRelations(localRelation.queryExecution.withCachedData) == 1) } test("SPARK-19093 Caching in side subquery") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index f4df80fd9c93f..6a190b98ea983 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.Matchers._ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.plans.logical.{Filter, OneRowRelation, Project, Union} +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{FilterExec, QueryExecution} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchange} @@ -869,6 +870,30 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(testData.select($"*").filter($"key" < 0).showString(1) === expectedAnswer) } + test("SPARK-18350 show with session local timezone") { + val d = Date.valueOf("2016-12-01") + val ts = Timestamp.valueOf("2016-12-01 00:00:00") + val df = Seq((d, ts)).toDF("d", "ts") + val expectedAnswer = """+----------+-------------------+ + ||d |ts | + |+----------+-------------------+ + ||2016-12-01|2016-12-01 00:00:00| + |+----------+-------------------+ + |""".stripMargin + assert(df.showString(1, truncate = 0) === expectedAnswer) + + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { + + val expectedAnswer = """+----------+-------------------+ + ||d |ts | + |+----------+-------------------+ + ||2016-12-01|2016-12-01 08:00:00| + |+----------+-------------------+ + |""".stripMargin + assert(df.showString(1, truncate = 0) === expectedAnswer) + } + } + test("createDataFrame(RDD[Row], StructType) should convert UDTs (SPARK-6672)") { val rowRDD = sparkContext.parallelize(Seq(Row(new ExamplePoint(1.0, 2.0)))) val schema = StructType(Array(StructField("point", new ExamplePointUDT(), false))) @@ -1626,17 +1651,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(d.size == d.distinct.size) } - test("SPARK-17625: data source table in InMemoryCatalog should guarantee output consistency") { - val tableName = "tbl" - withTable(tableName) { - spark.range(10).select('id as 'i, 'id as 'j).write.saveAsTable(tableName) - val relation = spark.sessionState.catalog.lookupRelation(TableIdentifier(tableName)) - val expr = relation.resolve("i") - val qe = spark.sessionState.executePlan(Project(Seq(expr), relation)) - qe.assertAnalyzed() - } - } - private def verifyNullabilityInFilterExec( df: DataFrame, expr: String, @@ -1736,4 +1750,9 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val df = spark.createDataFrame(spark.sparkContext.makeRDD(rows), schema) assert(df.filter($"array1" === $"array2").count() == 1) } + + test("SPARK-17913: compare long and string type column may return confusing result") { + val df = Seq(123L -> "123", 19157170390056973L -> "19157170390056971").toDF("i", "j") + checkAnswer(df.select($"i" === $"j"), Row(true) :: Row(false) :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 731a28c237bae..b37bf131e8dce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -898,13 +898,6 @@ class DatasetSuite extends QueryTest with SharedSQLContext { (1, 2), (1, 1), (2, 1), (2, 2)) } - test("dropDuplicates should not change child plan output") { - val ds = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() - checkDataset( - ds.dropDuplicates("_1").select(ds("_1").as[String], ds("_2").as[Int]), - ("a", 1), ("b", 1)) - } - test("SPARK-16097: Encoders.tuple should handle null object correctly") { val enc = Encoders.tuple(Encoders.tuple(Encoders.STRING, Encoders.STRING), Encoders.STRING) val data = Seq((("a", "b"), "c"), (null, "d")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala index f0995ea1d0025..b9871afd59e4f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -87,6 +87,13 @@ class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { Row(1) :: Row(2) :: Row(3) :: Nil) } + test("single explode_outer") { + val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") + checkAnswer( + df.select(explode_outer('intList)), + Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) + } + test("single posexplode") { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") checkAnswer( @@ -94,6 +101,13 @@ class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Nil) } + test("single posexplode_outer") { + val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") + checkAnswer( + df.select(posexplode_outer('intList)), + Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Row(null, null) :: Nil) + } + test("explode and other columns") { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") @@ -110,6 +124,26 @@ class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { Row(1, Seq(1, 2, 3), 3) :: Nil) } + test("explode_outer and other columns") { + val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") + + checkAnswer( + df.select($"a", explode_outer('intList)), + Row(1, 1) :: + Row(1, 2) :: + Row(1, 3) :: + Row(2, null) :: + Nil) + + checkAnswer( + df.select($"*", explode_outer('intList)), + Row(1, Seq(1, 2, 3), 1) :: + Row(1, Seq(1, 2, 3), 2) :: + Row(1, Seq(1, 2, 3), 3) :: + Row(2, Seq(), null) :: + Nil) + } + test("aliased explode") { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") @@ -122,6 +156,18 @@ class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { Row(6) :: Nil) } + test("aliased explode_outer") { + val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") + + checkAnswer( + df.select(explode_outer('intList).as('int)).select('int), + Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) + + checkAnswer( + df.select(explode('intList).as('int)).select(sum('int)), + Row(6) :: Nil) + } + test("explode on map") { val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") @@ -130,6 +176,15 @@ class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { Row("a", "b")) } + test("explode_outer on map") { + val df = Seq((1, Map("a" -> "b")), (2, Map[String, String]()), + (3, Map("c" -> "d"))).toDF("a", "map") + + checkAnswer( + df.select(explode_outer('map)), + Row("a", "b") :: Row(null, null) :: Row("c", "d") :: Nil) + } + test("explode on map with aliases") { val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") @@ -138,6 +193,14 @@ class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { Row("a", "b")) } + test("explode_outer on map with aliases") { + val df = Seq((3, None), (1, Some(Map("a" -> "b")))).toDF("a", "map") + + checkAnswer( + df.select(explode_outer('map).as("key1" :: "value1" :: Nil)).select("key1", "value1"), + Row("a", "b") :: Row(null, null) :: Nil) + } + test("self join explode") { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") val exploded = df.select(explode('intList).as('i)) @@ -207,6 +270,19 @@ class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { Row(1) :: Row(2) :: Nil) } + test("inline_outer") { + val df = Seq((1, "2"), (3, "4"), (5, "6")).toDF("col1", "col2") + val df2 = df.select(when('col1 === 1, null).otherwise(array(struct('col1, 'col2))).as("col1")) + checkAnswer( + df2.selectExpr("inline(col1)"), + Row(3, "4") :: Row(5, "6") :: Nil + ) + checkAnswer( + df2.selectExpr("inline_outer(col1)"), + Row(null, null) :: Row(3, "4") :: Row(5, "6") :: Nil + ) + } + test("SPARK-14986: Outer lateral view with empty generate expression") { checkAnswer( sql("select nil from values 1 lateral view outer explode(array()) n as nil"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 890cc5b560d02..9c39b3c7f09bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.functions.{from_json, struct, to_json} import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{CalendarIntervalType, IntegerType, StructType} +import org.apache.spark.sql.types.{CalendarIntervalType, IntegerType, StructType, TimestampType} class JsonFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -105,6 +105,16 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { Row(Row(1)) :: Nil) } + test("from_json with option") { + val df = Seq("""{"time": "26/08/2015 18:00"}""").toDS() + val schema = new StructType().add("time", TimestampType) + val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm") + + checkAnswer( + df.select(from_json($"value", schema, options)), + Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))) + } + test("from_json missing columns") { val df = Seq("""{"a": 1}""").toDS() val schema = new StructType().add("b", IntegerType) @@ -131,6 +141,15 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { Row("""{"_1":1}""") :: Nil) } + test("to_json with option") { + val df = Seq(Tuple1(Tuple1(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))).toDF("a") + val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm") + + checkAnswer( + df.select(to_json($"a", options)), + Row("""{"_1":"26/08/2015 18:00"}""") :: Nil) + } + test("to_json unsupported type") { val df = Seq(Tuple1(Tuple1("interval -3 month 7 hours"))).toDF("a") .select(struct($"a._1".cast(CalendarIntervalType).as("a")).as("c")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 605dec4a1ef90..a77f92059876e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -982,6 +982,33 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.sessionState.conf.clear() } + test("SPARK-19218 SET command should show a result in a sorted order") { + val overrideConfs = sql("SET").collect() + sql(s"SET test.key3=1") + sql(s"SET test.key2=2") + sql(s"SET test.key1=3") + val result = sql("SET").collect() + assert(result === + (overrideConfs ++ Seq( + Row("test.key1", "3"), + Row("test.key2", "2"), + Row("test.key3", "1"))).sortBy(_.getString(0)) + ) + spark.sessionState.conf.clear() + } + + test("SPARK-19218 `SET -v` should not fail with null value configuration") { + import SQLConf._ + val confEntry = SQLConfigBuilder("spark.test").doc("doc").stringConf.createWithDefault(null) + + try { + val result = sql("SET -v").collect() + assert(result === result.sortBy(_.getString(0))) + } finally { + SQLConf.unregister(confEntry) + } + } + test("SET commands with illegal or inappropriate argument") { spark.sessionState.conf.clear() // Set negative mapred.reduce.tasks for automatically determining @@ -2501,12 +2528,40 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("should be able to resolve a persistent view") { - withTable("t1") { + withTable("t1", "t2") { withView("v1") { sql("CREATE TABLE `t1` USING parquet AS SELECT * FROM VALUES(1, 1) AS t1(a, b)") - sql("CREATE VIEW `v1` AS SELECT * FROM t1") - checkAnswer(spark.table("v1"), Row(1, 1)) + sql("CREATE TABLE `t2` USING parquet AS SELECT * FROM VALUES('a', 2, 1.0) AS t2(d, e, f)") + sql("CREATE VIEW `v1`(x, y) AS SELECT * FROM t1") + checkAnswer(spark.table("v1").orderBy("x"), Row(1, 1)) + + sql("ALTER VIEW `v1` AS SELECT * FROM t2") + checkAnswer(spark.table("v1").orderBy("f"), Row("a", 2, 1.0)) } } } + + test("SPARK-19059: read file based table whose name starts with underscore") { + withTable("_tbl") { + sql("CREATE TABLE `_tbl`(i INT) USING parquet") + sql("INSERT INTO `_tbl` VALUES (1), (2), (3)") + checkAnswer( sql("SELECT * FROM `_tbl`"), Row(1) :: Row(2) :: Row(3) :: Nil) + } + } + + test("SPARK-19334: check code injection is prevented") { + // The end of comment (*/) should be escaped. + val badQuery = + """|SELECT inline(array(cast(struct(1) AS + | struct<`= + | new Object() { + | {f();} + | public void f() {throw new RuntimeException("This exception is injected.");} + | public int x; + | }.x + | `:int>)))""".stripMargin.replaceAll("\n", "") + + checkAnswer(sql(badQuery), Row(1) :: Nil) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 1a4049fb339cb..91aecca537fb2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -163,7 +163,12 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { s"-- Number of queries: ${outputs.size}\n\n\n" + outputs.zipWithIndex.map{case (qr, i) => qr.toString(i)}.mkString("\n\n\n") + "\n" } - stringToFile(new File(testCase.resultFile), goldenOutput) + val resultFile = new File(testCase.resultFile); + val parent = resultFile.getParentFile(); + if (!parent.exists()) { + assert(parent.mkdirs(), "Could not create directory: " + parent) + } + stringToFile(resultFile, goldenOutput) } // Read back the golden file. @@ -223,7 +228,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } catch { case a: AnalysisException if a.plan.nonEmpty => // Do not output the logical plan tree which contains expression IDs. - (StructType(Seq.empty), Seq(a.getClass.getName, a.getSimpleMessage)) + // Also implement a crude way of masking expression IDs in the error message + // with a generic pattern "###". + (StructType(Seq.empty), + Seq(a.getClass.getName, a.getSimpleMessage.replaceAll("#\\d+", "#x"))) case NonFatal(e) => // If there is an exception, put the exception class followed by the message. (StructType(Seq.empty), Seq(e.getClass.getName, e.getMessage)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 2ef8b18c04612..25dbecb5894e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -263,12 +263,12 @@ class SubquerySuite extends QueryTest with SharedSQLContext { Row(1, 2.0) :: Row(1, 2.0) :: Nil) checkAnswer( - sql("select * from l where a not in (select c from t where b < d)"), - Row(1, 2.0) :: Row(1, 2.0) :: Row(3, 3.0) :: Nil) + sql("select * from l where (a, b) not in (select c, d from t) and a < 4"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Nil) // Empty sub-query checkAnswer( - sql("select * from l where a not in (select c from r where c > 10 and b < d)"), + sql("select * from l where (a, b) not in (select c, d from r where c > 10)"), Row(1, 2.0) :: Row(1, 2.0) :: Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index e8ccefa69a42d..ae6b2bc3753fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ @@ -248,4 +249,17 @@ class UDFSuite extends QueryTest with SharedSQLContext { sql("SELECT tmp.t.* FROM (SELECT testDataFunc(a, b) AS t from testData2) tmp").toDF(), testData2) } + + test("SPARK-19338 Provide identical names for UDFs in the EXPLAIN output") { + def explainStr(df: DataFrame): String = { + val explain = ExplainCommand(df.queryExecution.logical, extended = false) + val sparkPlan = spark.sessionState.executePlan(explain).executedPlan + sparkPlan.executeCollect().map(_.getString(0).trim).headOption.getOrElse("") + } + val udf1 = "myUdf1" + val udf2 = "myUdf2" + spark.udf.register(udf1, (n: Int) => { n + 1 }) + spark.udf.register(udf2, (n: Int) => { n * 1 }) + assert(explainStr(sql("SELECT myUdf1(myUdf2(1))")).contains(s"UDF:$udf1(UDF:$udf2(1))")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 474f17ff7afbc..ea4a8ee7ff728 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.functions._ @@ -71,6 +72,77 @@ object UDT { } +// object and classes to test SPARK-19311 + +// Trait/Interface for base type +sealed trait IExampleBaseType extends Serializable { + def field: Int +} + +// Trait/Interface for derived type +sealed trait IExampleSubType extends IExampleBaseType + +// a base class +class ExampleBaseClass(override val field: Int) extends IExampleBaseType + +// a derived class +class ExampleSubClass(override val field: Int) + extends ExampleBaseClass(field) with IExampleSubType + +// UDT for base class +class ExampleBaseTypeUDT extends UserDefinedType[IExampleBaseType] { + + override def sqlType: StructType = { + StructType(Seq( + StructField("intfield", IntegerType, nullable = false))) + } + + override def serialize(obj: IExampleBaseType): InternalRow = { + val row = new GenericInternalRow(1) + row.setInt(0, obj.field) + row + } + + override def deserialize(datum: Any): IExampleBaseType = { + datum match { + case row: InternalRow => + require(row.numFields == 1, + "ExampleBaseTypeUDT requires row with length == 1") + val field = row.getInt(0) + new ExampleBaseClass(field) + } + } + + override def userClass: Class[IExampleBaseType] = classOf[IExampleBaseType] +} + +// UDT for derived class +private[spark] class ExampleSubTypeUDT extends UserDefinedType[IExampleSubType] { + + override def sqlType: StructType = { + StructType(Seq( + StructField("intfield", IntegerType, nullable = false))) + } + + override def serialize(obj: IExampleSubType): InternalRow = { + val row = new GenericInternalRow(1) + row.setInt(0, obj.field) + row + } + + override def deserialize(datum: Any): IExampleSubType = { + datum match { + case row: InternalRow => + require(row.numFields == 1, + "ExampleSubTypeUDT requires row with length == 1") + val field = row.getInt(0) + new ExampleSubClass(field) + } + } + + override def userClass: Class[IExampleSubType] = classOf[IExampleSubType] +} + class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetTest { import testImplicits._ @@ -194,4 +266,35 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT // call `collect` to make sure this query can pass analysis. pointsRDD.as[MyLabeledPoint].map(_.copy(label = 2.0)).collect() } + + test("SPARK-19311: UDFs disregard UDT type hierarchy") { + UDTRegistration.register(classOf[IExampleBaseType].getName, + classOf[ExampleBaseTypeUDT].getName) + UDTRegistration.register(classOf[IExampleSubType].getName, + classOf[ExampleSubTypeUDT].getName) + + // UDF that returns a base class object + sqlContext.udf.register("doUDF", (param: Int) => { + new ExampleBaseClass(param) + }: IExampleBaseType) + + // UDF that returns a derived class object + sqlContext.udf.register("doSubTypeUDF", (param: Int) => { + new ExampleSubClass(param) + }: IExampleSubType) + + // UDF that takes a base class object as parameter + sqlContext.udf.register("doOtherUDF", (obj: IExampleBaseType) => { + obj.field + }: Int) + + // this worked already before the fix SPARK-19311: + // return type of doUDF equals parameter type of doOtherUDF + sql("SELECT doOtherUDF(doUDF(41))") + + // this one passes only with the fix SPARK-19311: + // return type of doSubUDF is a subtype of the parameter type of doOtherUDF + sql("SELECT doOtherUDF(doSubTypeUDF(42))") + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index ac3878e849051..51f5946c19e00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.command import java.io.File +import java.net.URI import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach @@ -96,7 +97,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { .add("col2", "string") .add("a", "int") .add("b", "int"), - provider = Some("hive"), + provider = Some("parquet"), partitionColumnNames = Seq("a", "b"), createTime = 0L, tracksPartitionsInCatalog = true) @@ -758,7 +759,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { testUnsetProperties(isDatasourceTable = true) } - test("alter table: set serde") { + // TODO: move this test to HiveDDLSuite.scala + ignore("alter table: set serde") { testSetSerde(isDatasourceTable = false) } @@ -766,7 +768,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { testSetSerde(isDatasourceTable = true) } - test("alter table: set serde partition") { + // TODO: move this test to HiveDDLSuite.scala + ignore("alter table: set serde partition") { testSetSerdePartition(isDatasourceTable = false) } @@ -1479,49 +1482,31 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { ) } - test("select/insert into the managed table") { + test("create a managed Hive source table") { assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") val tabName = "tbl" withTable(tabName) { - sql(s"CREATE TABLE $tabName (i INT, j STRING)") - val catalogTable = - spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) - assert(catalogTable.tableType == CatalogTableType.MANAGED) - - var message = intercept[AnalysisException] { - sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1, 'a'") - }.getMessage - assert(message.contains("Hive support is required to insert into the following tables")) - message = intercept[AnalysisException] { - sql(s"SELECT * FROM $tabName") + val e = intercept[AnalysisException] { + sql(s"CREATE TABLE $tabName (i INT, j STRING)") }.getMessage - assert(message.contains("Hive support is required to select over the following tables")) + assert(e.contains("Hive support is required to CREATE Hive TABLE")) } } - test("select/insert into external table") { + test("create an external Hive source table") { assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") withTempDir { tempDir => val tabName = "tbl" withTable(tabName) { - sql( - s""" - |CREATE EXTERNAL TABLE $tabName (i INT, j STRING) - |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' - |LOCATION '$tempDir' + val e = intercept[AnalysisException] { + sql( + s""" + |CREATE EXTERNAL TABLE $tabName (i INT, j STRING) + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |LOCATION '${tempDir.toURI}' """.stripMargin) - val catalogTable = - spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) - assert(catalogTable.tableType == CatalogTableType.EXTERNAL) - - var message = intercept[AnalysisException] { - sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1, 'a'") - }.getMessage - assert(message.contains("Hive support is required to insert into the following tables")) - message = intercept[AnalysisException] { - sql(s"SELECT * FROM $tabName") }.getMessage - assert(message.contains("Hive support is required to select over the following tables")) + assert(e.contains("Hive support is required to CREATE Hive TABLE")) } } } @@ -1582,7 +1567,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql("USE temp") sql("DROP DATABASE temp") val e = intercept[AnalysisException] { - sql("CREATE TABLE t (a INT, b INT)") + sql("CREATE TABLE t (a INT, b INT) USING parquet") }.getMessage assert(e.contains("Database 'temp' not found")) } @@ -1692,20 +1677,27 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("truncate table - external table, temporary table, view (not allowed)") { import testImplicits._ - val path = Utils.createTempDir().getAbsolutePath - (1 to 10).map { i => (i, i) }.toDF("a", "b").createTempView("my_temp_tab") - sql(s"CREATE EXTERNAL TABLE my_ext_tab LOCATION '$path'") - sql(s"CREATE VIEW my_view AS SELECT 1") - intercept[NoSuchTableException] { - sql("TRUNCATE TABLE my_temp_tab") + withTempPath { tempDir => + withTable("my_ext_tab") { + (("a", "b") :: Nil).toDF().write.parquet(tempDir.getCanonicalPath) + (1 to 10).map { i => (i, i) }.toDF("a", "b").createTempView("my_temp_tab") + sql(s"CREATE TABLE my_ext_tab using parquet LOCATION '${tempDir.toURI}'") + sql(s"CREATE VIEW my_view AS SELECT 1") + intercept[NoSuchTableException] { + sql("TRUNCATE TABLE my_temp_tab") + } + assertUnsupported("TRUNCATE TABLE my_ext_tab") + assertUnsupported("TRUNCATE TABLE my_view") + } } - assertUnsupported("TRUNCATE TABLE my_ext_tab") - assertUnsupported("TRUNCATE TABLE my_view") } test("truncate table - non-partitioned table (not allowed)") { - sql("CREATE TABLE my_tab (age INT, name STRING)") - assertUnsupported("TRUNCATE TABLE my_tab PARTITION (age=10)") + withTable("my_tab") { + sql("CREATE TABLE my_tab (age INT, name STRING) using parquet") + sql("INSERT INTO my_tab values (10, 'a')") + assertUnsupported("TRUNCATE TABLE my_tab PARTITION (age=10)") + } } test("SPARK-16034 Partition columns should match when appending to existing data source tables") { @@ -1787,4 +1779,32 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val rows: Seq[Row] = df.toLocalIterator().asScala.toSeq assert(rows.length > 0) } + + test("SET LOCATION for managed table") { + withTable("tbl") { + withTempDir { dir => + sql("CREATE TABLE tbl(i INT) USING parquet") + sql("INSERT INTO tbl SELECT 1") + checkAnswer(spark.table("tbl"), Row(1)) + val defaultTablePath = spark.sessionState.catalog + .getTableMetadata(TableIdentifier("tbl")).storage.locationUri.get + + sql(s"ALTER TABLE tbl SET LOCATION '${dir.getCanonicalPath}'") + spark.catalog.refreshTable("tbl") + // SET LOCATION won't move data from previous table path to new table path. + assert(spark.table("tbl").count() == 0) + // the previous table path should be still there. + assert(new File(new URI(defaultTablePath)).exists()) + + sql("INSERT INTO tbl SELECT 2") + checkAnswer(spark.table("tbl"), Row(2)) + // newly inserted data will go to the new table path. + assert(dir.listFiles().nonEmpty) + + sql("DROP TABLE tbl") + // the new table path will be removed after DROP TABLE. + assert(!dir.exists()) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParserSuite.scala similarity index 77% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParserSuite.scala index ffd3d260bcb40..2ca6308852a7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParserSuite.scala @@ -25,7 +25,9 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -class CSVTypeCastSuite extends SparkFunSuite { +class UnivocityParserSuite extends SparkFunSuite { + private val parser = + new UnivocityParser(StructType(Seq.empty), new CSVOptions(Map.empty[String, String])) private def assertNull(v: Any) = assert(v == null) @@ -36,7 +38,7 @@ class CSVTypeCastSuite extends SparkFunSuite { stringValues.zip(decimalValues).foreach { case (strVal, decimalVal) => val decimalValue = new BigDecimal(decimalVal.toString) - assert(CSVTypeCast.makeConverter("_1", decimalType).apply(strVal) === + assert(parser.makeConverter("_1", decimalType).apply(strVal) === Decimal(decimalValue, decimalType.precision, decimalType.scale)) } } @@ -73,19 +75,19 @@ class CSVTypeCastSuite extends SparkFunSuite { types.foreach { t => // Tests that a custom nullValue. val converter = - CSVTypeCast.makeConverter("_1", t, nullable = true, CSVOptions("nullValue", "-")) + parser.makeConverter("_1", t, nullable = true, CSVOptions("nullValue", "-")) assertNull(converter.apply("-")) assertNull(converter.apply(null)) // Tests that the default nullValue is empty string. - assertNull(CSVTypeCast.makeConverter("_1", t, nullable = true).apply("")) + assertNull(parser.makeConverter("_1", t, nullable = true).apply("")) } // Not nullable field with nullValue option. types.foreach { t => // Casts a null to not nullable field should throw an exception. val converter = - CSVTypeCast.makeConverter("_1", t, nullable = false, CSVOptions("nullValue", "-")) + parser.makeConverter("_1", t, nullable = false, CSVOptions("nullValue", "-")) var message = intercept[RuntimeException] { converter.apply("-") }.getMessage @@ -100,32 +102,32 @@ class CSVTypeCastSuite extends SparkFunSuite { // null. Seq(true, false).foreach { b => val converter = - CSVTypeCast.makeConverter("_1", StringType, nullable = b, CSVOptions("nullValue", "null")) + parser.makeConverter("_1", StringType, nullable = b, CSVOptions("nullValue", "null")) assert(converter.apply("") == UTF8String.fromString("")) } } test("Throws exception for empty string with non null type") { val exception = intercept[RuntimeException]{ - CSVTypeCast.makeConverter("_1", IntegerType, nullable = false, CSVOptions()).apply("") + parser.makeConverter("_1", IntegerType, nullable = false, CSVOptions()).apply("") } assert(exception.getMessage.contains("null value found but field _1 is not nullable.")) } test("Types are cast correctly") { - assert(CSVTypeCast.makeConverter("_1", ByteType).apply("10") == 10) - assert(CSVTypeCast.makeConverter("_1", ShortType).apply("10") == 10) - assert(CSVTypeCast.makeConverter("_1", IntegerType).apply("10") == 10) - assert(CSVTypeCast.makeConverter("_1", LongType).apply("10") == 10) - assert(CSVTypeCast.makeConverter("_1", FloatType).apply("1.00") == 1.0) - assert(CSVTypeCast.makeConverter("_1", DoubleType).apply("1.00") == 1.0) - assert(CSVTypeCast.makeConverter("_1", BooleanType).apply("true") == true) + assert(parser.makeConverter("_1", ByteType).apply("10") == 10) + assert(parser.makeConverter("_1", ShortType).apply("10") == 10) + assert(parser.makeConverter("_1", IntegerType).apply("10") == 10) + assert(parser.makeConverter("_1", LongType).apply("10") == 10) + assert(parser.makeConverter("_1", FloatType).apply("1.00") == 1.0) + assert(parser.makeConverter("_1", DoubleType).apply("1.00") == 1.0) + assert(parser.makeConverter("_1", BooleanType).apply("true") == true) val timestampsOptions = CSVOptions("timestampFormat", "dd/MM/yyyy hh:mm") val customTimestamp = "31/01/2015 00:00" val expectedTime = timestampsOptions.timestampFormat.parse(customTimestamp).getTime val castedTimestamp = - CSVTypeCast.makeConverter("_1", TimestampType, nullable = true, timestampsOptions) + parser.makeConverter("_1", TimestampType, nullable = true, timestampsOptions) .apply(customTimestamp) assert(castedTimestamp == expectedTime * 1000L) @@ -133,14 +135,14 @@ class CSVTypeCastSuite extends SparkFunSuite { val dateOptions = CSVOptions("dateFormat", "dd/MM/yyyy") val expectedDate = dateOptions.dateFormat.parse(customDate).getTime val castedDate = - CSVTypeCast.makeConverter("_1", DateType, nullable = true, dateOptions) + parser.makeConverter("_1", DateType, nullable = true, dateOptions) .apply(customTimestamp) assert(castedDate == DateTimeUtils.millisToDays(expectedDate)) val timestamp = "2015-01-01 00:00:00" - assert(CSVTypeCast.makeConverter("_1", TimestampType).apply(timestamp) == + assert(parser.makeConverter("_1", TimestampType).apply(timestamp) == DateTimeUtils.stringToTime(timestamp).getTime * 1000L) - assert(CSVTypeCast.makeConverter("_1", DateType).apply("2015-01-01") == + assert(parser.makeConverter("_1", DateType).apply("2015-01-01") == DateTimeUtils.millisToDays(DateTimeUtils.stringToTime("2015-01-01").getTime)) } @@ -149,15 +151,15 @@ class CSVTypeCastSuite extends SparkFunSuite { try { Locale.setDefault(new Locale("fr", "FR")) // Would parse as 1.0 in fr-FR - assert(CSVTypeCast.makeConverter("_1", FloatType).apply("1,00") == 100.0) - assert(CSVTypeCast.makeConverter("_1", DoubleType).apply("1,00") == 100.0) + assert(parser.makeConverter("_1", FloatType).apply("1,00") == 100.0) + assert(parser.makeConverter("_1", DoubleType).apply("1,00") == 100.0) } finally { Locale.setDefault(originalLocale) } } test("Float NaN values are parsed correctly") { - val floatVal: Float = CSVTypeCast.makeConverter( + val floatVal: Float = parser.makeConverter( "_1", FloatType, nullable = true, CSVOptions("nanValue", "nn") ).apply("nn").asInstanceOf[Float] @@ -167,7 +169,7 @@ class CSVTypeCastSuite extends SparkFunSuite { } test("Double NaN values are parsed correctly") { - val doubleVal: Double = CSVTypeCast.makeConverter( + val doubleVal: Double = parser.makeConverter( "_1", DoubleType, nullable = true, CSVOptions("nanValue", "-") ).apply("-").asInstanceOf[Double] @@ -175,13 +177,13 @@ class CSVTypeCastSuite extends SparkFunSuite { } test("Float infinite values can be parsed") { - val floatVal1 = CSVTypeCast.makeConverter( + val floatVal1 = parser.makeConverter( "_1", FloatType, nullable = true, CSVOptions("negativeInf", "max") ).apply("max").asInstanceOf[Float] assert(floatVal1 == Float.NegativeInfinity) - val floatVal2 = CSVTypeCast.makeConverter( + val floatVal2 = parser.makeConverter( "_1", FloatType, nullable = true, CSVOptions("positiveInf", "max") ).apply("max").asInstanceOf[Float] @@ -189,13 +191,13 @@ class CSVTypeCastSuite extends SparkFunSuite { } test("Double infinite values can be parsed") { - val doubleVal1 = CSVTypeCast.makeConverter( + val doubleVal1 = parser.makeConverter( "_1", DoubleType, nullable = true, CSVOptions("negativeInf", "max") ).apply("max").asInstanceOf[Double] assert(doubleVal1 == Double.NegativeInfinity) - val doubleVal2 = CSVTypeCast.makeConverter( + val doubleVal2 = parser.makeConverter( "_1", DoubleType, nullable = true, CSVOptions("positiveInf", "max") ).apply("max").asInstanceOf[Double] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index a324183b43748..161a409d83ad4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -448,7 +448,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // Number and String conflict: resolve the type as number in this query. checkAnswer( - sql("select num_str + 1.2 from jsonTable where num_str > 14"), + sql("select num_str + 1.2 from jsonTable where num_str > 14d"), Row(92233720368547758071.2) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala index 81bea2fef8bd4..2a3d1cf0b298a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.api.python.PythonFunction -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, GreaterThan, In} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, Expression, GreaterThan, In} import org.apache.spark.sql.execution.{FilterExec, InputAdapter, SparkPlanTest, WholeStageCodegenExec} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.BooleanType @@ -86,13 +86,11 @@ class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { test("Python UDF refers to the attributes from more than one child") { val df = Seq(("Hello", 4)).toDF("a", "b") val df2 = Seq(("Hello", 4)).toDF("c", "d") - val joinDF = df.join(df2).where("dummyPythonUDF(a, c) == dummyPythonUDF(d, c)") - - val e = intercept[RuntimeException] { - joinDF.queryExecution.executedPlan - }.getMessage - assert(Seq("Invalid PythonUDF dummyUDF", "requires attributes from more than one child") - .forall(e.contains)) + val joinDF = df.crossJoin(df2).where("dummyPythonUDF(a, c) == dummyPythonUDF(d, c)") + val qualifiedPlanNodes = joinDF.queryExecution.executedPlan.collect { + case b: BatchEvalPythonExec => b + } + assert(qualifiedPlanNodes.size == 1) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index 5dd04543ed144..75723d0abcfcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.internal +import java.io.File +import java.net.URI + import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite @@ -27,7 +30,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types.StructType /** @@ -37,12 +40,14 @@ class CatalogSuite extends SparkFunSuite with BeforeAndAfterEach with SharedSQLContext { + import testImplicits._ private def sessionCatalog: SessionCatalog = spark.sessionState.catalog private val utils = new CatalogTestUtils { override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat" override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat" + override val defaultProvider: String = "parquet" override def newEmptyCatalog(): ExternalCatalog = spark.sharedState.externalCatalog } @@ -306,22 +311,6 @@ class CatalogSuite columnFields.foreach { f => assert(columnString.contains(f.toString)) } } - test("createExternalTable should fail if path is not given for file-based data source") { - val e = intercept[AnalysisException] { - spark.catalog.createExternalTable("tbl", "json", Map.empty[String, String]) - } - assert(e.message.contains("Unable to infer schema")) - - val e2 = intercept[AnalysisException] { - spark.catalog.createExternalTable( - "tbl", - "json", - new StructType().add("i", IntegerType), - Map.empty[String, String]) - } - assert(e2.message == "Cannot create a file-based external data source table without path") - } - test("dropTempView should not un-cache and drop metastore table if a same-name table exists") { withTable("same_name") { spark.range(10).write.saveAsTable("same_name") @@ -460,6 +449,50 @@ class CatalogSuite } } + test("createTable with 'path' in options") { + withTable("t") { + withTempDir { dir => + spark.catalog.createTable( + tableName = "t", + source = "json", + schema = new StructType().add("i", "int"), + options = Map("path" -> dir.getAbsolutePath)) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.tableType == CatalogTableType.EXTERNAL) + assert(table.storage.locationUri.get == dir.getAbsolutePath) + + Seq((1)).toDF("i").write.insertInto("t") + assert(dir.exists() && dir.listFiles().nonEmpty) + + sql("DROP TABLE t") + // the table path and data files are still there after DROP TABLE, if custom table path is + // specified. + assert(dir.exists() && dir.listFiles().nonEmpty) + } + } + } + + test("createTable without 'path' in options") { + withTable("t") { + spark.catalog.createTable( + tableName = "t", + source = "json", + schema = new StructType().add("i", "int"), + options = Map.empty[String, String]) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.tableType == CatalogTableType.MANAGED) + val tablePath = new File(new URI(table.storage.locationUri.get)) + assert(tablePath.exists() && tablePath.listFiles().isEmpty) + + Seq((1)).toDF("i").write.insertInto("t") + assert(tablePath.listFiles().nonEmpty) + + sql("DROP TABLE t") + // the table path is removed after DROP TABLE, if custom table path is not specified. + assert(!tablePath.exists()) + } + } + // TODO: add tests for the rest of them } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala index 953604e4ac417..bf7fabe33266b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala @@ -19,11 +19,31 @@ package org.apache.spark.sql.sources import java.io.File +import org.apache.hadoop.mapreduce.TaskAttemptContext + +import org.apache.spark.internal.Logging import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils +private class OnlyDetectCustomPathFileCommitProtocol(jobId: String, path: String, isAppend: Boolean) + extends SQLHadoopMapReduceCommitProtocol(jobId, path, isAppend) + with Serializable with Logging { + + override def newTaskTempFileAbsPath( + taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { + if (isAppend) { + throw new Exception("append data to an existed partitioned table, " + + "there should be no custom partition path sent to Task") + } + + super.newTaskTempFileAbsPath(taskContext, absoluteDir, ext) + } +} + class PartitionedWriteSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -92,6 +112,18 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { } } + test("append data to an existed partitioned table without custom partition path") { + withTable("t") { + withSQLConf(SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[OnlyDetectCustomPathFileCommitProtocol].getName) { + Seq((1, 2)).toDF("a", "b").write.partitionBy("b").saveAsTable("t") + // if custom partition path is detected by the task, it will throw an Exception + // from OnlyDetectCustomPathFileCommitProtocol above. + Seq((3, 2)).toDF("a", "b").write.mode("append").partitionBy("b").saveAsTable("t") + } + } + } + /** Lists files recursively. */ private def recursiveList(f: File): Array[File] = { require(f.isDirectory) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MapGroupsWithStateSuite.scala new file mode 100644 index 0000000000000..e712b41904ad8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MapGroupsWithStateSuite.scala @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkException +import org.apache.spark.sql.State +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ +import org.apache.spark.sql.execution.StateImpl +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.state.StateStore + +/** Class to check custom state types */ +case class RunningCount(count: Long) + +class MapGroupsWithStateSuite extends StreamTest with BeforeAndAfterAll { + + import testImplicits._ + + override def afterAll(): Unit = { + super.afterAll() + StateStore.stop() + } + + test("state - get, exists, update, remove") { + var state: StateImpl[String] = null + + def testState( + expectedData: Option[String], + shouldBeUpdated: Boolean = false, + shouldBeRemoved: Boolean = false + ): Unit = { + if (expectedData.isDefined) { + assert(state.exists) + assert(state.get() === expectedData.get) + assert(state.getOption() === expectedData) + } else { + assert(!state.exists) + intercept[NoSuchElementException] { + state.get() + } + assert(state.getOption() === None) + } + + assert(state.isUpdated === shouldBeUpdated) + assert(state.isRemoved === shouldBeRemoved) + } + + // Updating empty state + state = StateImpl[String](None) + testState(None) + state.update("") + testState(Some(""), shouldBeUpdated = true) + + // Updating exiting state, even if with null + state = StateImpl[String](Some("2")) + testState(Some("2")) + state.update("3") + testState(Some("3"), shouldBeUpdated = true) + state.update(null) + testState(Some(null), shouldBeUpdated = true) + + // Removing state + state.remove() + testState(None, shouldBeRemoved = true, shouldBeUpdated = false) + state.remove() // should be still callable + state.update("4") + testState(Some("4"), shouldBeRemoved = false, shouldBeUpdated = true) + } + + test("flatMapGroupsWithState - streaming") { + // Function to maintain running count up to 2, and then remove the count + // Returns the data and the count if state is defined, otherwise does not return anything + val stateFunc = (key: String, values: Iterator[String], state: State[RunningCount]) => { + + var count = state.getOption.map(_.count).getOrElse(0L) + values.size + if (count == 3) { + state.remove() + Iterator.empty + } else { + state.update(RunningCount(count)) + Iterator((key, count.toString)) + } + } + + val inputData = MemoryStream[String] + val result = + inputData.toDS() + .groupByKey(x => x) + .flatMapGroupsWithState(stateFunc) // State: Int, Out: (Str, Str) + + testStream(result, Append)( + AddData(inputData, "a"), + CheckLastBatch(("a", "1")), + assertNumStateRows(total = 1, updated = 1), + AddData(inputData, "a", "b"), + CheckLastBatch(("a", "2"), ("b", "1")), + assertNumStateRows(total = 2, updated = 2), + StopStream, + StartStream(), + AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a + CheckLastBatch(("b", "2")), + assertNumStateRows(total = 1, updated = 2), + StopStream, + StartStream(), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and + CheckLastBatch(("a", "1"), ("c", "1")), + assertNumStateRows(total = 3, updated = 2) + ) + } + + test("flatMapGroupsWithState - batch") { + // Function that returns running count only if its even, otherwise does not return + val stateFunc = (key: String, values: Iterator[String], state: State[RunningCount]) => { + if (state.exists) throw new IllegalArgumentException("state.exists should be false") + if (state.getOption.nonEmpty) { + throw new IllegalArgumentException("state.getOption should be empty") + } + Iterator((key, values.size)) + } + checkAnswer( + Seq("a", "a", "b").toDS.groupByKey(x => x).flatMapGroupsWithState(stateFunc).toDF, + Seq(("a", 2), ("b", 1)).toDF) + } + + test("mapGroupsWithState - streaming") { + // Function to maintain running count up to 2, and then remove the count + // Returns the data and the count (-1 if count reached beyond 2 and state was just removed) + val stateFunc = (key: String, values: Iterator[String], state: State[RunningCount]) => { + + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + if (count == 3) { + state.remove() + (key, "-1") + } else { + state.update(RunningCount(count)) + (key, count.toString) + } + } + + val inputData = MemoryStream[String] + val result = + inputData.toDS() + .groupByKey(x => x) + .mapGroupsWithState(stateFunc) // Types = State: MyState, Out: (Str, Str) + + testStream(result, Append)( + AddData(inputData, "a"), + CheckLastBatch(("a", "1")), + assertNumStateRows(total = 1, updated = 1), + AddData(inputData, "a", "b"), + CheckLastBatch(("a", "2"), ("b", "1")), + assertNumStateRows(total = 2, updated = 2), + StopStream, + StartStream(), + AddData(inputData, "a", "b"), // should remove state for "a" and return count as -1 + CheckLastBatch(("a", "-1"), ("b", "2")), + assertNumStateRows(total = 1, updated = 2), + StopStream, + StartStream(), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 + CheckLastBatch(("a", "1"), ("c", "1")), + assertNumStateRows(total = 3, updated = 2) + ) + } + + test("mapGroupsWithState - batch") { + val stateFunc = (key: String, values: Iterator[String], state: State[RunningCount]) => { + if (state.exists) throw new IllegalArgumentException("state.exists should be false") + if (state.getOption.nonEmpty) { + throw new IllegalArgumentException("state.getOption should be empty") + } + (key, values.size) + } + + checkAnswer( + spark.createDataset(Seq("a", "a", "b")) + .groupByKey(x => x) + .mapGroupsWithState(stateFunc) + .toDF, + spark.createDataset(Seq(("a", 2), ("b", 1))).toDF) + } + + testQuietly("StateStore.abort on task failure handling") { + val stateFunc = (key: String, values: Iterator[String], state: State[RunningCount]) => { + if (MapGroupsWithStateSuite.failInTask) throw new Exception("expected failure") + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + (key, count) + } + + val inputData = MemoryStream[String] + val result = + inputData.toDS() + .groupByKey(x => x) + .mapGroupsWithState(stateFunc) // Types = State: MyState, Out: (Str, Str) + + def setFailInTask(value: Boolean): AssertOnQuery = AssertOnQuery { q => + MapGroupsWithStateSuite.failInTask = value + true + } + + testStream(result, Append)( + setFailInTask(false), + AddData(inputData, "a"), + CheckLastBatch(("a", 1L)), + AddData(inputData, "a"), + CheckLastBatch(("a", 2L)), + setFailInTask(true), + AddData(inputData, "a"), + ExpectFailure[SparkException](), // task should fail but should not increment count + setFailInTask(false), + StartStream(), + CheckLastBatch(("a", 3L)) // task should not fail, and should show correct count + ) + } + + private def assertNumStateRows(total: Long, updated: Long): AssertOnQuery = AssertOnQuery { q => + val progressWithData = q.recentProgress.filter(_.numInputRows > 0).lastOption.get + assert(progressWithData.stateOperators(0).numRowsTotal === total) + assert(progressWithData.stateOperators(0).numRowsUpdated === updated) + true + } +} + +object MapGroupsWithStateSuite { + var failInTask = true +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index e964e646d22aa..f31dc8add48d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -304,6 +304,32 @@ class StreamSuite extends StreamTest { q.stop() } } + + test("SPARK-19065: dropDuplicates should not create expressions using the same id") { + withTempPath { testPath => + val data = Seq((1, 2), (2, 3), (3, 4)) + data.toDS.write.mode("overwrite").json(testPath.getCanonicalPath) + val schema = spark.read.json(testPath.getCanonicalPath).schema + val query = spark + .readStream + .schema(schema) + .json(testPath.getCanonicalPath) + .dropDuplicates("_1") + .writeStream + .format("memory") + .queryName("testquery") + .outputMode("complete") + .start() + try { + query.processAllAvailable() + if (query.exception.isDefined) { + throw query.exception.get + } + } finally { + query.stop() + } + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 4aa4100522cde..af2f31a34d8da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -385,7 +385,12 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { .streamingQuery // Wait until the initialization finishes, because some tests need to use `logicalPlan` // after starting the query. - currentStream.awaitInitialization(streamingTimeout.toMillis) + try { + currentStream.awaitInitialization(streamingTimeout.toMillis) + } catch { + case _: StreamingQueryException => + // Ignore the exception. `StopStream` or `ExpectFailure` will catch it as well. + } case AdvanceManualClock(timeToAdd) => verify(currentStream != null, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala index 8e16fd418a37c..f05e9d1fda73f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala @@ -30,8 +30,9 @@ import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException -import org.apache.spark.sql.Dataset +import org.apache.spark.sql.{AnalysisException, Dataset} import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.BlockingSource import org.apache.spark.util.Utils @@ -238,6 +239,15 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { } } + test("SPARK-19268: Adaptive query execution should be disallowed") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val e = intercept[AnalysisException] { + MemoryStream[Int].toDS.writeStream.queryName("test-query").format("memory").start() + } + assert(e.getMessage.contains(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) && + e.getMessage.contains("not supported")) + } + } /** Run a body of code by defining a query on each dataset */ private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[StreamingQuery] => Unit): Unit = { diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index d217e9b4feb6d..f78660f7c14b6 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -41,6 +41,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning private val originalConvertMetastoreOrc = TestHive.sessionState.convertMetastoreOrc private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled + private val originalSessionLocalTimeZone = TestHive.conf.sessionLocalTimeZone def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -62,6 +63,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, false) // Ensures that cross joins are enabled so that we can test them TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) + // Fix session local timezone to America/Los_Angeles for those timezone sensitive tests + // (timestamp_*) + TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, "America/Los_Angeles") RuleExecutor.resetTime() } @@ -74,6 +78,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, originalConvertMetastoreOrc) TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) + TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, originalSessionLocalTimeZone) // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) diff --git a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 32aa13ff257a6..e7d762fbebe76 100644 --- a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1 +1,2 @@ org.apache.spark.sql.hive.orc.OrcFileFormat +org.apache.spark.sql.hive.execution.HiveFileFormat \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index e4b1f6ae3e49e..faa76b73fde4b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -17,17 +17,15 @@ package org.apache.spark.sql.hive -import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import com.google.common.util.concurrent.Striped import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.orc.OrcFileFormat @@ -41,9 +39,7 @@ import org.apache.spark.sql.types._ */ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Logging { private val sessionState = sparkSession.sessionState.asInstanceOf[HiveSessionState] - - /** A fully qualified identifier for a table (i.e., database.tableName) */ - case class QualifiedTableName(database: String, name: String) + private lazy val tableRelationCache = sparkSession.sessionState.catalog.tableRelationCache private def getCurrentDatabase: String = sessionState.catalog.getCurrentDatabase @@ -65,45 +61,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } } - /** A cache of Spark SQL data source tables that have been accessed. */ - protected[hive] val cachedDataSourceTables: LoadingCache[QualifiedTableName, LogicalPlan] = { - val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() { - override def load(in: QualifiedTableName): LogicalPlan = { - logDebug(s"Creating new cached data source for $in") - val table = sparkSession.sharedState.externalCatalog.getTable(in.database, in.name) - - val pathOption = table.storage.locationUri.map("path" -> _) - val dataSource = - DataSource( - sparkSession, - // In older version(prior to 2.1) of Spark, the table schema can be empty and should be - // inferred at runtime. We should still support it. - userSpecifiedSchema = if (table.schema.isEmpty) None else Some(table.schema), - partitionColumns = table.partitionColumnNames, - bucketSpec = table.bucketSpec, - className = table.provider.get, - options = table.storage.properties ++ pathOption, - catalogTable = Some(table)) - - LogicalRelation(dataSource.resolveRelation(), catalogTable = Some(table)) - } - } - - CacheBuilder.newBuilder().maximumSize(1000).build(cacheLoader) - } - - def refreshTable(tableIdent: TableIdentifier): Unit = { - // refreshTable does not eagerly reload the cache. It just invalidate the cache. - // Next time when we use the table, it will be populated in the cache. - // Since we also cache ParquetRelations converted from Hive Parquet tables and - // adding converted ParquetRelations into the cache is not defined in the load function - // of the cache (instead, we add the cache entry in convertToParquetRelation), - // it is better at here to invalidate the cache to avoid confusing waring logs from the - // cache loader (e.g. cannot find data source provider, which is only defined for - // data source table.). - cachedDataSourceTables.invalidate(getQualifiedTableName(tableIdent)) - } - def hiveDefaultTableFilePath(tableIdent: TableIdentifier): String = { // Code based on: hiveWarehouse.getTablePath(currentDatabase, tableName) val QualifiedTableName(dbName, tblName) = getQualifiedTableName(tableIdent) @@ -111,45 +68,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log new Path(new Path(dbLocation), tblName).toString } - /** - * Returns a [[LogicalPlan]] that represents the given table or view from Hive metastore. - * - * @param tableIdent The name of the table/view that we look up. - * @param alias The alias name of the table/view that we look up. - * @return a [[LogicalPlan]] that represents the given table or view from Hive metastore. - */ - def lookupRelation( - tableIdent: TableIdentifier, - alias: Option[String]): LogicalPlan = { - val qualifiedTableName = getQualifiedTableName(tableIdent) - val table = sparkSession.sharedState.externalCatalog.getTable( - qualifiedTableName.database, qualifiedTableName.name) - - if (DDLUtils.isDatasourceTable(table)) { - val dataSourceTable = cachedDataSourceTables(qualifiedTableName) - val qualifiedTable = SubqueryAlias(qualifiedTableName.name, dataSourceTable, None) - // Then, if alias is specified, wrap the table with a Subquery using the alias. - // Otherwise, wrap the table with a Subquery using the table name. - alias.map(a => SubqueryAlias(a, qualifiedTable, None)).getOrElse(qualifiedTable) - } else if (table.tableType == CatalogTableType.VIEW) { - val tableIdentifier = table.identifier - val viewText = table.viewText.getOrElse(sys.error("Invalid view without text.")) - // The relation is a view, so we wrap the relation by: - // 1. Add a [[View]] operator over the relation to keep track of the view desc; - // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the name of the view. - val child = View( - desc = table, - output = table.schema.toAttributes, - child = sparkSession.sessionState.sqlParser.parsePlan(viewText)) - SubqueryAlias(alias.getOrElse(tableIdentifier.table), child, Option(tableIdentifier)) - } else { - val qualifiedTable = - MetastoreRelation( - qualifiedTableName.database, qualifiedTableName.name)(table, sparkSession) - alias.map(a => SubqueryAlias(a, qualifiedTable, None)).getOrElse(qualifiedTable) - } - } - private def getCached( tableIdentifier: QualifiedTableName, pathsInMetastore: Seq[Path], @@ -159,7 +77,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log expectedBucketSpec: Option[BucketSpec], partitionSchema: Option[StructType]): Option[LogicalRelation] = { - cachedDataSourceTables.getIfPresent(tableIdentifier) match { + tableRelationCache.getIfPresent(tableIdentifier) match { case null => None // Cache miss case logical @ LogicalRelation(relation: HadoopFsRelation, _, _) => val cachedRelationFileFormatClass = relation.fileFormat.getClass @@ -178,7 +96,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log Some(logical) } else { // If the cached relation is not updated, we invalidate it right away. - cachedDataSourceTables.invalidate(tableIdentifier) + tableRelationCache.invalidate(tableIdentifier) None } case _ => @@ -187,7 +105,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log s"should be stored as $expectedFileFormat. However, we are getting " + s"a ${relation.fileFormat} from the metastore cache. This cached " + s"entry will be invalidated.") - cachedDataSourceTables.invalidate(tableIdentifier) + tableRelationCache.invalidate(tableIdentifier) None } case other => @@ -195,7 +113,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} should be stored " + s"as $expectedFileFormat. However, we are getting a $other from the metastore cache. " + s"This cached entry will be invalidated.") - cachedDataSourceTables.invalidate(tableIdentifier) + tableRelationCache.invalidate(tableIdentifier) None } } @@ -270,7 +188,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val created = LogicalRelation(relation, catalogTable = Some(metastoreRelation.catalogTable)) - cachedDataSourceTables.put(tableIdentifier, created) + tableRelationCache.put(tableIdentifier, created) created } @@ -298,7 +216,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log className = fileType).resolveRelation(), catalogTable = Some(metastoreRelation.catalogTable)) - cachedDataSourceTables.put(tableIdentifier, created) + tableRelationCache.put(tableIdentifier, created) created } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index b3cbbedbe1ee3..44ef5cce2ee05 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -27,12 +27,12 @@ import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, Gener import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchTableException} +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.catalog.{FunctionResourceLoader, GlobalTempViewManager, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.parser.ParserInterface -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.internal.SQLConf @@ -58,28 +58,6 @@ private[sql] class HiveSessionCatalog( hadoopConf, parser) { - override def lookupRelation(name: TableIdentifier, alias: Option[String] = None): LogicalPlan = { - synchronized { - val table = formatTableName(name.table) - val db = formatDatabaseName(name.database.getOrElse(currentDb)) - if (db == globalTempViewManager.database) { - val relationAlias = alias.getOrElse(table) - globalTempViewManager.get(table).map { viewDef => - SubqueryAlias(relationAlias, viewDef, Some(name)) - }.getOrElse(throw new NoSuchTableException(db, table)) - } else if (name.database.isDefined || !tempTables.contains(table)) { - val newName = name.copy(database = Some(db), table = table) - metastoreCatalog.lookupRelation(newName, alias) - } else { - val relation = tempTables(table) - val tableWithQualifiers = SubqueryAlias(table, relation, None) - // If an alias was specified by the lookup, wrap the plan in a subquery so that - // attributes are properly qualified with this alias. - alias.map(a => SubqueryAlias(a, tableWithQualifiers, None)).getOrElse(tableWithQualifiers) - } - } - } - // ---------------------------------------------------------------- // | Methods and fields for interacting with HiveMetastoreCatalog | // ---------------------------------------------------------------- @@ -93,15 +71,6 @@ private[sql] class HiveSessionCatalog( val ParquetConversions: Rule[LogicalPlan] = metastoreCatalog.ParquetConversions val OrcConversions: Rule[LogicalPlan] = metastoreCatalog.OrcConversions - override def refreshTable(name: TableIdentifier): Unit = { - super.refreshTable(name) - metastoreCatalog.refreshTable(name) - } - - def invalidateCache(): Unit = { - metastoreCatalog.cachedDataSourceTables.invalidateAll() - } - def hiveDefaultTableFilePath(name: TableIdentifier): String = { metastoreCatalog.hiveDefaultTableFilePath(name) } @@ -109,7 +78,7 @@ private[sql] class HiveSessionCatalog( // For testing only private[hive] def getCachedDataSourceTable(table: TableIdentifier): LogicalPlan = { val key = metastoreCatalog.getQualifiedTableName(table) - metastoreCatalog.cachedDataSourceTables.getIfPresent(key) + sparkSession.sessionState.catalog.tableRelationCache.getIfPresent(key) } override def makeFunctionBuilder(funcName: String, className: String): FunctionBuilder = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 9b4b8b6fcd910..9fd03ef8ba037 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -62,11 +62,16 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) override val extendedResolutionRules = catalog.ParquetConversions :: catalog.OrcConversions :: + new DetermineHiveSerde(conf) :: + new FindDataSourceTable(sparkSession) :: + new FindHiveSerdeTable(sparkSession) :: + new ResolveDataSource(sparkSession) :: Nil + + override val postHocResolutionRules = AnalyzeCreateTable(sparkSession) :: PreprocessTableInsertion(conf) :: DataSourceAnalysis(conf) :: - new DetermineHiveSerde(conf) :: - new ResolveDataSource(sparkSession) :: Nil + new HiveAnalysis(sparkSession) :: Nil override val extendedCheckRules = Seq(PreWriteCheck(conf, catalog)) } @@ -88,7 +93,6 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) SpecialLimits, InMemoryScans, HiveTableScans, - DataSinks, Scripts, Aggregation, JoinSelection, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index d1f11e78b4ea2..badccae16837b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -18,15 +18,14 @@ package org.apache.spark.sql.hive import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, SimpleCatalogRelation} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, ScriptTransformation} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.command.{DDLUtils, ExecutedCommandExec} -import org.apache.spark.sql.execution.datasources.CreateTable +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.execution.datasources.{CreateTable, PreprocessTableInsertion} import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -40,14 +39,6 @@ class DetermineHiveSerde(conf: SQLConf) extends Rule[LogicalPlan] { if (t.bucketSpec.isDefined) { throw new AnalysisException("Creating bucketed Hive serde table is not supported yet.") } - if (t.partitionColumnNames.nonEmpty && query.isDefined) { - val errorMessage = "A Create Table As Select (CTAS) statement is not allowed to " + - "create a partitioned table using Hive's file formats. " + - "Please use the syntax of \"CREATE TABLE tableName USING dataSource " + - "OPTIONS (...) PARTITIONED BY ...\" to create a partitioned table through a " + - "CTAS statement." - throw new AnalysisException(errorMessage) - } val defaultStorage = HiveSerDe.getDefaultStorage(conf) val options = new HiveOptions(t.storage.properties) @@ -86,6 +77,40 @@ class DetermineHiveSerde(conf: SQLConf) extends Rule[LogicalPlan] { } } +/** + * Replaces generic operations with specific variants that are designed to work with Hive. + * + * Note that, this rule must be run after [[PreprocessTableInsertion]]. + */ +class HiveAnalysis(session: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case InsertIntoTable(table: MetastoreRelation, partSpec, query, overwrite, ifNotExists) => + InsertIntoHiveTable(table, partSpec, query, overwrite, ifNotExists) + + case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => + val dbName = tableDesc.identifier.database.getOrElse(session.catalog.currentDatabase) + CreateHiveTableAsSelectCommand( + tableDesc.copy(identifier = tableDesc.identifier.copy(database = Some(dbName))), + query, + mode) + } +} + +/** + * Replaces `SimpleCatalogRelation` with [[MetastoreRelation]] if its table provider is hive. + */ +class FindHiveSerdeTable(session: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case i @ InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) + if DDLUtils.isHiveTable(s.metadata) => + i.copy(table = + MetastoreRelation(s.metadata.database, s.metadata.identifier.table)(s.metadata, session)) + + case s: SimpleCatalogRelation if DDLUtils.isHiveTable(s.metadata) => + MetastoreRelation(s.metadata.database, s.metadata.identifier.table)(s.metadata, session) + } +} + private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. self: SparkPlanner => @@ -94,35 +119,9 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.ScriptTransformation(input, script, output, child, ioschema) => + case ScriptTransformation(input, script, output, child, ioschema) => val hiveIoSchema = HiveScriptIOSchema(ioschema) - ScriptTransformation(input, script, output, planLater(child), hiveIoSchema) :: Nil - case _ => Nil - } - } - - object DataSinks extends Strategy { - def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.InsertIntoTable( - table: MetastoreRelation, partition, child, overwrite, ifNotExists) => - InsertIntoHiveTable( - table, partition, planLater(child), overwrite, ifNotExists) :: Nil - - case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => - // Currently `DataFrameWriter.saveAsTable` doesn't support - // the Append mode of hive serde tables yet. - if (mode == SaveMode.Append) { - throw new AnalysisException( - "CTAS for hive serde tables does not support append semantics.") - } - - val dbName = tableDesc.identifier.database.getOrElse(sparkSession.catalog.currentDatabase) - val cmd = CreateHiveTableAsSelectCommand( - tableDesc.copy(identifier = tableDesc.identifier.copy(database = Some(dbName))), - query, - mode == SaveMode.Ignore) - ExecutedCommandExec(cmd) :: Nil - + ScriptTransformationExec(input, script, output, planLater(child), hiveIoSchema) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 7254f73f41f37..346757c2047a7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -36,7 +36,6 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeMap, AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.execution.FileRelation -import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.types.StructField diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index aaf30f41f29c2..b4b63032ab261 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -311,10 +311,10 @@ private[hive] object HiveTableUtil { // that calls Hive.get() which tries to access metastore, but it's not valid in runtime // it would be fixed in next version of hive but till then, we should use this instead def configureJobPropertiesForStorageHandler( - tableDesc: TableDesc, jobConf: JobConf, input: Boolean) { + tableDesc: TableDesc, conf: Configuration, input: Boolean) { val property = tableDesc.getProperties.getProperty(META_TABLE_STORAGE) val storageHandler = - org.apache.hadoop.hive.ql.metadata.HiveUtils.getStorageHandler(jobConf, property) + org.apache.hadoop.hive.ql.metadata.HiveUtils.getStorageHandler(conf, property) if (storageHandler != null) { val jobProperties = new java.util.LinkedHashMap[String, String] if (input) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 5c0e2f6ec4941..bf703a5ab6e60 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -426,7 +426,9 @@ private[hive] class HiveClientImpl( // in the function toHiveTable. properties = properties.filter(kv => kv._1 != "comment" && kv._1 != "EXTERNAL"), comment = properties.get("comment"), - viewOriginalText = Option(h.getViewOriginalText), + // In older versions of Spark(before 2.2.0), we expand the view original text and store + // that into `viewExpandedText`, and that should be used in view resolution. So we get + // `viewExpandedText` instead of `viewOriginalText` for viewText here. viewText = Option(h.getViewExpandedText), unsupportedFeatures = unsupportedFeatures) } @@ -471,6 +473,7 @@ private[hive] class HiveClientImpl( // do the check at first and collect all the matching partitions val matchingParts = specs.flatMap { s => + assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid") // The provided spec here can be a partial spec, i.e. it will match all partitions // whose specs are supersets of this partial spec. E.g. If a table has partitions // (b='1', c='1') and (b='1', c='2'), a partial spec of (b='1') will match both. @@ -545,6 +548,7 @@ private[hive] class HiveClientImpl( // -1 for result limit means "no limit/return all" client.getPartitionNames(table.database, table.identifier.table, -1) case Some(s) => + assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid") client.getPartitionNames(table.database, table.identifier.table, s.asJava, -1) } hivePartitionNames.asScala.sorted @@ -568,7 +572,9 @@ private[hive] class HiveClientImpl( val hiveTable = toHiveTable(table) val parts = spec match { case None => shim.getAllPartitions(client, hiveTable).map(fromHivePartition) - case Some(s) => client.getPartitions(hiveTable, s.asJava).asScala.map(fromHivePartition) + case Some(s) => + assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid") + client.getPartitions(hiveTable, s.asJava).asScala.map(fromHivePartition) } HiveCatalogMetrics.incrementFetchedPartitions(parts.length) parts @@ -850,8 +856,13 @@ private[hive] class HiveClientImpl( table.storage.properties.foreach { case (k, v) => hiveTable.setSerdeParam(k, v) } table.properties.foreach { case (k, v) => hiveTable.setProperty(k, v) } table.comment.foreach { c => hiveTable.setProperty("comment", c) } - table.viewOriginalText.foreach { t => hiveTable.setViewOriginalText(t) } - table.viewText.foreach { t => hiveTable.setViewExpandedText(t) } + // Hive will expand the view text, so it needs 2 fields: viewOriginalText and viewExpandedText. + // Since we don't expand the view text, but only add table properties, we map the `viewText` to + // the both fields in hive table. + table.viewText.foreach { t => + hiveTable.setViewOriginalText(t) + hiveTable.setViewExpandedText(t) + } hiveTable } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 26b2de8e768f5..63fdd6b090e6f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -122,6 +122,7 @@ private[hive] object IsolatedClientLoader extends Logging { // TODO: Remove copy logic. val tempDir = Utils.createTempDir(namePrefix = s"hive-${version}") allFiles.foreach(f => FileUtils.copyFileToDirectory(f, tempDir)) + logInfo(s"Downloaded metastore jars to ${tempDir.getCanonicalPath}") tempDir.listFiles().map(_.toURI.toURL) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala index b1b8439efa011..4e2193b6abc3f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive /** Support for interacting with different versions of the HiveMetastoreClient */ package object client { - private[client] abstract class HiveVersion( + private[hive] abstract class HiveVersion( val fullVersion: String, val extraDeps: Seq[String] = Nil, val exclusions: Seq[String] = Nil) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index ef5a5a001fb6f..41c6b18e9d794 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.hive.execution import scala.util.control.NonFatal -import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.hive.MetastoreRelation /** @@ -31,13 +31,12 @@ import org.apache.spark.sql.hive.MetastoreRelation * * @param tableDesc the Table Describe, which may contains serde, storage handler etc. * @param query the query whose result will be insert into the new relation - * @param ignoreIfExists allow continue working if it's already exists, otherwise - * raise exception + * @param mode SaveMode */ case class CreateHiveTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, - ignoreIfExists: Boolean) + mode: SaveMode) extends RunnableCommand { private val tableIdentifier = tableDesc.identifier @@ -45,50 +44,41 @@ case class CreateHiveTableAsSelectCommand( override def innerChildren: Seq[LogicalPlan] = Seq(query) override def run(sparkSession: SparkSession): Seq[Row] = { - lazy val metastoreRelation: MetastoreRelation = { - import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe - import org.apache.hadoop.io.Text - import org.apache.hadoop.mapred.TextInputFormat - - val withFormat = - tableDesc.withNewStorage( - inputFormat = - tableDesc.storage.inputFormat.orElse(Some(classOf[TextInputFormat].getName)), - outputFormat = - tableDesc.storage.outputFormat - .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), - serde = tableDesc.storage.serde.orElse(Some(classOf[LazySimpleSerDe].getName)), - compressed = tableDesc.storage.compressed) - - val withSchema = if (withFormat.schema.isEmpty) { - // Hive doesn't support specifying the column list for target table in CTAS - // However we don't think SparkSQL should follow that. - tableDesc.copy(schema = query.output.toStructType) - } else { - withFormat - } - - sparkSession.sessionState.catalog.createTable(withSchema, ignoreIfExists = false) - - // Get the Metastore Relation - sparkSession.sessionState.catalog.lookupRelation(tableIdentifier) match { - case r: MetastoreRelation => r - } - } - // TODO ideally, we should get the output data ready first and then - // add the relation into catalog, just in case of failure occurs while data - // processing. if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) { - if (ignoreIfExists) { - // table already exists, will do nothing, to keep consistent with Hive - } else { + assert(mode != SaveMode.Overwrite, + s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") + + if (mode == SaveMode.ErrorIfExists) { throw new AnalysisException(s"$tableIdentifier already exists.") } + if (mode == SaveMode.Ignore) { + // Since the table already exists and the save mode is Ignore, we will just return. + return Seq.empty + } + + sparkSession.sessionState.executePlan( + InsertIntoTable( + UnresolvedRelation(tableIdentifier), + Map(), + query, + overwrite = false, + ifNotExists = false)).toRdd } else { + // TODO ideally, we should get the output data ready first and then + // add the relation into catalog, just in case of failure occurs while data + // processing. + assert(tableDesc.schema.isEmpty) + sparkSession.sessionState.catalog.createTable( + tableDesc.copy(schema = query.schema), ignoreIfExists = false) + try { - sparkSession.sessionState.executePlan(InsertIntoTable( - metastoreRelation, Map(), query, overwrite = true, ifNotExists = false)).toRdd + sparkSession.sessionState.executePlan( + InsertIntoTable( + UnresolvedRelation(tableIdentifier), + Map(), + query, + overwrite = true, + ifNotExists = false)).toRdd } catch { case NonFatal(e) => // drop the created table. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala new file mode 100644 index 0000000000000..ac735e8b383f6 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.serde2.Serializer +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, StructObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.{JobConf, Reporter} +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory} +import org.apache.spark.sql.hive.{HiveInspectors, HiveTableUtil} +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableJobConf + +/** + * `FileFormat` for writing Hive tables. + * + * TODO: implement the read logic. + */ +class HiveFileFormat(fileSinkConf: FileSinkDesc) + extends FileFormat with DataSourceRegister with Logging { + + def this() = this(null) + + override def shortName(): String = "hive" + + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + throw new UnsupportedOperationException(s"inferSchema is not supported for hive data source.") + } + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val conf = job.getConfiguration + val tableDesc = fileSinkConf.getTableInfo + conf.set("mapred.output.format.class", tableDesc.getOutputFileFormatClassName) + + // When speculation is on and output committer class name contains "Direct", we should warn + // users that they may loss data if they are using a direct output committer. + val speculationEnabled = sparkSession.sparkContext.conf.getBoolean("spark.speculation", false) + val outputCommitterClass = conf.get("mapred.output.committer.class", "") + if (speculationEnabled && outputCommitterClass.contains("Direct")) { + val warningMessage = + s"$outputCommitterClass may be an output committer that writes data directly to " + + "the final location. Because speculation is enabled, this output committer may " + + "cause data loss (see the case in SPARK-10063). If possible, please use an output " + + "committer that does not have this behavior (e.g. FileOutputCommitter)." + logWarning(warningMessage) + } + + // Add table properties from storage handler to hadoopConf, so any custom storage + // handler settings can be set to hadoopConf + HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, conf, false) + Utilities.copyTableJobPropertiesToConf(tableDesc, conf) + + // Avoid referencing the outer object. + val fileSinkConfSer = fileSinkConf + new OutputWriterFactory { + private val jobConf = new SerializableJobConf(new JobConf(conf)) + @transient private lazy val outputFormat = + jobConf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]] + + override def getFileExtension(context: TaskAttemptContext): String = { + Utilities.getFileExtension(jobConf.value, fileSinkConfSer.getCompressed, outputFormat) + } + + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new HiveOutputWriter(path, fileSinkConfSer, jobConf.value, dataSchema) + } + } + } +} + +class HiveOutputWriter( + path: String, + fileSinkConf: FileSinkDesc, + jobConf: JobConf, + dataSchema: StructType) extends OutputWriter with HiveInspectors { + + private def tableDesc = fileSinkConf.getTableInfo + + private val serializer = { + val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] + serializer.initialize(null, tableDesc.getProperties) + serializer + } + + private val hiveWriter = HiveFileFormatUtils.getHiveRecordWriter( + jobConf, + tableDesc, + serializer.getSerializedClass, + fileSinkConf, + new Path(path), + Reporter.NULL) + + private val standardOI = ObjectInspectorUtils + .getStandardObjectInspector( + tableDesc.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + private val fieldOIs = + standardOI.getAllStructFieldRefs.asScala.map(_.getFieldObjectInspector).toArray + private val dataTypes = dataSchema.map(_.dataType).toArray + private val wrappers = fieldOIs.zip(dataTypes).map { case (f, dt) => wrapperFor(f, dt) } + private val outputData = new Array[Any](fieldOIs.length) + + override def write(row: InternalRow): Unit = { + var i = 0 + while (i < fieldOIs.length) { + outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) + i += 1 + } + hiveWriter.write(serializer.serialize(outputData, standardOI)) + } + + override def close(): Unit = { + // Seems the boolean value passed into close does not matter. + hiveWriter.close(false) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index 7ee5fc543c55b..def6ef3691333 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -146,9 +146,19 @@ case class HiveTableScanExec( hadoopReader.makeRDDForTable(relation.hiveQlTable) } } else { + // The attribute name of predicate could be different than the one in schema in case of + // case insensitive, we should change them to match the one in schema, so we do not need to + // worry about case sensitivity anymore. + val normalizedFilters = partitionPruningPred.map { e => + e transform { + case a: AttributeReference => + a.withName(relation.output.find(_.semanticEquals(a)).get.name) + } + } + Utils.withDummyCallSite(sqlContext.sparkContext) { hadoopReader.makeRDDForPartitionedTable( - prunePartitions(relation.getHiveQlPartitions(partitionPruningPred))) + prunePartitions(relation.getHiveQlPartitions(normalizedFilters))) } } val numOutputRows = longMetric("numOutputRows") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index aa858e808edf7..ce418ae135dd9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -24,22 +24,22 @@ import java.util.{Date, Locale, Random} import scala.util.control.NonFatal +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.exec.TaskRunner import org.apache.hadoop.hive.ql.ErrorMsg -import org.apache.hadoop.mapred.{FileOutputFormat, JobConf} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.{AnalysisException, Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormatWriter import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.hive.client.HiveVersion import org.apache.spark.SparkException -import org.apache.spark.util.SerializableJobConf /** @@ -69,26 +69,20 @@ import org.apache.spark.util.SerializableJobConf * {{{ * Map('a' -> Some('1'), 'b' -> None) * }}}. - * @param child the logical plan representing data to write to. + * @param query the logical plan representing data to write to. * @param overwrite overwrite existing table or partitions. * @param ifNotExists If true, only write if the table or partition does not exist. */ case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], - child: SparkPlan, + query: LogicalPlan, overwrite: Boolean, - ifNotExists: Boolean) extends UnaryExecNode { + ifNotExists: Boolean) extends RunnableCommand { - @transient private val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] - @transient private val externalCatalog = sqlContext.sharedState.externalCatalog + override protected def innerChildren: Seq[LogicalPlan] = query :: Nil - def output: Seq[Attribute] = Seq.empty - - val hadoopConf = sessionState.newHadoopConf() var createdTempDir: Option[Path] = None - val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") - val scratchDir = hadoopConf.get("hive.exec.scratchdir", "/tmp/hive") private def executionId: String = { val rand: Random = new Random @@ -96,7 +90,10 @@ case class InsertIntoHiveTable( "hive_" + format.format(new Date) + "_" + Math.abs(rand.nextLong) } - private def getStagingDir(inputPath: Path): Path = { + private def getStagingDir( + inputPath: Path, + hadoopConf: Configuration, + stagingDir: String): Path = { val inputPathUri: URI = inputPath.toUri val inputPathName: String = inputPathUri.getPath val fs: FileSystem = inputPath.getFileSystem(hadoopConf) @@ -121,17 +118,27 @@ case class InsertIntoHiveTable( throw new RuntimeException( "Cannot create staging directory '" + dir.toString + "': " + e.getMessage, e) } - return dir + dir } - private def getExternalScratchDir(extURI: URI): Path = { - getStagingDir(new Path(extURI.getScheme, extURI.getAuthority, extURI.getPath)) + private def getExternalScratchDir( + extURI: URI, + hadoopConf: Configuration, + stagingDir: String): Path = { + getStagingDir( + new Path(extURI.getScheme, extURI.getAuthority, extURI.getPath), + hadoopConf, + stagingDir) } - def getExternalTmpPath(path: Path): Path = { + def getExternalTmpPath( + path: Path, + hiveVersion: HiveVersion, + hadoopConf: Configuration, + stagingDir: String, + scratchDir: String): Path = { import org.apache.spark.sql.hive.client.hive._ - val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version // Before Hive 1.1, when inserting into a table, Hive will create the staging directory under // a common scratch directory. After the writing is finished, Hive will simply empty the table // directory and move the staging directory to it. @@ -142,16 +149,19 @@ case class InsertIntoHiveTable( // staging directory under the table director for Hive prior to 1.1, the staging directory will // be removed by Hive when Hive is trying to empty the table directory. if (hiveVersion == v12 || hiveVersion == v13 || hiveVersion == v14 || hiveVersion == v1_0) { - oldVersionExternalTempPath(path) + oldVersionExternalTempPath(path, hadoopConf, scratchDir) } else if (hiveVersion == v1_1 || hiveVersion == v1_2) { - newVersionExternalTempPath(path) + newVersionExternalTempPath(path, hadoopConf, stagingDir) } else { throw new IllegalStateException("Unsupported hive version: " + hiveVersion.fullVersion) } } // Mostly copied from Context.java#getExternalTmpPath of Hive 0.13 - def oldVersionExternalTempPath(path: Path): Path = { + def oldVersionExternalTempPath( + path: Path, + hadoopConf: Configuration, + scratchDir: String): Path = { val extURI: URI = path.toUri val scratchPath = new Path(scratchDir, executionId) var dirPath = new Path( @@ -176,54 +186,44 @@ case class InsertIntoHiveTable( } // Mostly copied from Context.java#getExternalTmpPath of Hive 1.2 - def newVersionExternalTempPath(path: Path): Path = { + def newVersionExternalTempPath( + path: Path, + hadoopConf: Configuration, + stagingDir: String): Path = { val extURI: URI = path.toUri if (extURI.getScheme == "viewfs") { - getExtTmpPathRelTo(path.getParent) + getExtTmpPathRelTo(path.getParent, hadoopConf, stagingDir) } else { - new Path(getExternalScratchDir(extURI), "-ext-10000") + new Path(getExternalScratchDir(extURI, hadoopConf, stagingDir), "-ext-10000") } } - def getExtTmpPathRelTo(path: Path): Path = { - new Path(getStagingDir(path), "-ext-10000") // Hive uses 10000 - } - - private def saveAsHiveFile( - rdd: RDD[InternalRow], - valueClass: Class[_], - fileSinkConf: FileSinkDesc, - conf: SerializableJobConf, - writerContainer: SparkHiveWriterContainer): Unit = { - assert(valueClass != null, "Output value class not set") - conf.value.setOutputValueClass(valueClass) - - val outputFileFormatClassName = fileSinkConf.getTableInfo.getOutputFileFormatClassName - assert(outputFileFormatClassName != null, "Output format class not set") - conf.value.set("mapred.output.format.class", outputFileFormatClassName) - - FileOutputFormat.setOutputPath( - conf.value, - SparkHiveWriterContainer.createPathFromString(fileSinkConf.getDirName(), conf.value)) - log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) - writerContainer.driverSideSetup() - sqlContext.sparkContext.runJob(rdd, writerContainer.writeToFile _) - writerContainer.commitJob() + def getExtTmpPathRelTo( + path: Path, + hadoopConf: Configuration, + stagingDir: String): Path = { + new Path(getStagingDir(path, hadoopConf, stagingDir), "-ext-10000") // Hive uses 10000 } /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the * `org.apache.hadoop.hive.serde2.SerDe` and the * `org.apache.hadoop.mapred.OutputFormat` provided by the table definition. - * - * Note: this is run once and then kept to avoid double insertions. */ - protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { + override def run(sparkSession: SparkSession): Seq[Row] = { + val sessionState = sparkSession.sessionState + val externalCatalog = sparkSession.sharedState.externalCatalog + val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version + val hadoopConf = sessionState.newHadoopConf() + val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") + val scratchDir = hadoopConf.get("hive.exec.scratchdir", "/tmp/hive") + // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc val tableLocation = table.hiveQlTable.getDataLocation - val tmpLocation = getExternalTmpPath(tableLocation) + val tmpLocation = + getExternalTmpPath(tableLocation, hiveVersion, hadoopConf, stagingDir, scratchDir) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) val isCompressed = hadoopConf.get("hive.exec.compress.output", "false").toBoolean @@ -276,40 +276,31 @@ case class InsertIntoHiveTable( } } - val jobConf = new JobConf(hadoopConf) - val jobConfSer = new SerializableJobConf(jobConf) - - // When speculation is on and output committer class name contains "Direct", we should warn - // users that they may loss data if they are using a direct output committer. - val speculationEnabled = sqlContext.sparkContext.conf.getBoolean("spark.speculation", false) - val outputCommitterClass = jobConf.get("mapred.output.committer.class", "") - if (speculationEnabled && outputCommitterClass.contains("Direct")) { - val warningMessage = - s"$outputCommitterClass may be an output committer that writes data directly to " + - "the final location. Because speculation is enabled, this output committer may " + - "cause data loss (see the case in SPARK-10063). If possible, please use an output " + - "committer that does not have this behavior (e.g. FileOutputCommitter)." - logWarning(warningMessage) + val committer = FileCommitProtocol.instantiate( + sparkSession.sessionState.conf.fileCommitProtocolClass, + jobId = java.util.UUID.randomUUID().toString, + outputPath = tmpLocation.toString, + isAppend = false) + + val partitionAttributes = partitionColumnNames.takeRight(numDynamicPartitions).map { name => + query.resolve(name :: Nil, sparkSession.sessionState.analyzer.resolver).getOrElse { + throw new AnalysisException( + s"Unable to resolve $name given [${query.output.map(_.name).mkString(", ")}]") + }.asInstanceOf[Attribute] } - val writerContainer = if (numDynamicPartitions > 0) { - val dynamicPartColNames = partitionColumnNames.takeRight(numDynamicPartitions) - new SparkHiveDynamicPartitionWriterContainer( - jobConf, - fileSinkConf, - dynamicPartColNames, - child.output) - } else { - new SparkHiveWriterContainer( - jobConf, - fileSinkConf, - child.output) - } - - @transient val outputClass = writerContainer.newSerializer(table.tableDesc).getSerializedClass - saveAsHiveFile(child.execute(), outputClass, fileSinkConf, jobConfSer, writerContainer) + FileFormatWriter.write( + sparkSession = sparkSession, + queryExecution = Dataset.ofRows(sparkSession, query).queryExecution, + fileFormat = new HiveFileFormat(fileSinkConf), + committer = committer, + outputSpec = FileFormatWriter.OutputSpec(tmpLocation.toString, Map.empty), + hadoopConf = hadoopConf, + partitionColumns = partitionAttributes, + bucketSpec = None, + refreshFunction = _ => (), + options = Map.empty) - val outputPath = FileOutputFormat.getOutputPath(jobConf) // TODO: Correctly set holdDDLTime. // In most of the time, we should have holdDDLTime = false. // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. @@ -319,7 +310,7 @@ case class InsertIntoHiveTable( externalCatalog.loadDynamicPartitions( db = table.catalogTable.database, table = table.catalogTable.identifier.table, - outputPath.toString, + tmpLocation.toString, partitionSpec, overwrite, numDynamicPartitions, @@ -363,7 +354,7 @@ case class InsertIntoHiveTable( externalCatalog.loadPartition( table.catalogTable.database, table.catalogTable.identifier.table, - outputPath.toString, + tmpLocation.toString, partitionSpec, isOverwrite = doHiveOverwrite, holdDDLTime = holdDDLTime, @@ -375,7 +366,7 @@ case class InsertIntoHiveTable( externalCatalog.loadTable( table.catalogTable.database, table.catalogTable.identifier.table, - outputPath.toString, // TODO: URI + tmpLocation.toString, // TODO: URI overwrite, holdDDLTime, isSrcLocal = false) @@ -391,21 +382,13 @@ case class InsertIntoHiveTable( } // Invalidate the cache. - sqlContext.sharedState.cacheManager.invalidateCache(table) - sqlContext.sessionState.catalog.refreshTable(table.catalogTable.identifier) + sparkSession.sharedState.cacheManager.invalidateCache(table) + sparkSession.sessionState.catalog.refreshTable(table.catalogTable.identifier) // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which // does not return anything for insert operations. // TODO: implement hive compatibility as rules. - Seq.empty[InternalRow] - } - - override def outputPartitioning: Partitioning = child.outputPartitioning - - override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray - - protected override def doExecute(): RDD[InternalRow] = { - sqlContext.sparkContext.parallelize(sideEffectResult.asInstanceOf[Seq[InternalRow]], 1) + Seq.empty[Row] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala similarity index 99% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala index 50855e48bc8fe..e7c165c5f86c5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala @@ -52,7 +52,7 @@ import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfig * @param script the command that should be executed. * @param output the attributes that are produced by the script. */ -case class ScriptTransformation( +case class ScriptTransformationExec( input: Seq[Expression], script: String, output: Seq[Attribute], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala deleted file mode 100644 index 0c9321068c4c1..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ /dev/null @@ -1,356 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.text.NumberFormat -import java.util.{Date, Locale} - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.common.FileUtils -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} -import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} -import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.serde2.Serializer -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, StructObjectInspector} -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred._ -import org.apache.hadoop.mapreduce.TaskType - -import org.apache.spark._ -import org.apache.spark.internal.Logging -import org.apache.spark.internal.io.SparkHadoopWriterUtils -import org.apache.spark.mapred.SparkHadoopMapRedUtil -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.UnsafeKVExternalSorter -import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} -import org.apache.spark.sql.types._ -import org.apache.spark.util.SerializableJobConf -import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter - -/** - * Internal helper class that saves an RDD using a Hive OutputFormat. - * It is based on `SparkHadoopWriter`. - */ -private[hive] class SparkHiveWriterContainer( - @transient private val jobConf: JobConf, - fileSinkConf: FileSinkDesc, - inputSchema: Seq[Attribute]) - extends Logging - with HiveInspectors - with Serializable { - - private val now = new Date() - private val tableDesc: TableDesc = fileSinkConf.getTableInfo - // Add table properties from storage handler to jobConf, so any custom storage - // handler settings can be set to jobConf - if (tableDesc != null) { - HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, jobConf, false) - Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) - } - protected val conf = new SerializableJobConf(jobConf) - - private var jobID = 0 - private var splitID = 0 - private var attemptID = 0 - private var jID: SerializableWritable[JobID] = null - private var taID: SerializableWritable[TaskAttemptID] = null - - @transient private var writer: FileSinkOperator.RecordWriter = null - @transient protected lazy val committer = conf.value.getOutputCommitter - @transient protected lazy val jobContext = new JobContextImpl(conf.value, jID.value) - @transient private lazy val taskContext = new TaskAttemptContextImpl(conf.value, taID.value) - @transient private lazy val outputFormat = - conf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]] - - def driverSideSetup() { - setIDs(0, 0, 0) - setConfParams() - committer.setupJob(jobContext) - } - - def executorSideSetup(jobId: Int, splitId: Int, attemptId: Int) { - setIDs(jobId, splitId, attemptId) - setConfParams() - committer.setupTask(taskContext) - initWriters() - } - - protected def getOutputName: String = { - val numberFormat = NumberFormat.getInstance(Locale.US) - numberFormat.setMinimumIntegerDigits(5) - numberFormat.setGroupingUsed(false) - val extension = Utilities.getFileExtension(conf.value, fileSinkConf.getCompressed, outputFormat) - "part-" + numberFormat.format(splitID) + extension - } - - def close() { - // Seems the boolean value passed into close does not matter. - if (writer != null) { - writer.close(false) - commit() - } - } - - def commitJob() { - committer.commitJob(jobContext) - } - - protected def initWriters() { - // NOTE this method is executed at the executor side. - // For Hive tables without partitions or with only static partitions, only 1 writer is needed. - writer = HiveFileFormatUtils.getHiveRecordWriter( - conf.value, - fileSinkConf.getTableInfo, - conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], - fileSinkConf, - FileOutputFormat.getTaskOutputPath(conf.value, getOutputName), - Reporter.NULL) - } - - protected def commit() { - SparkHadoopMapRedUtil.commitTask(committer, taskContext, jobID, splitID) - } - - def abortTask(): Unit = { - if (committer != null) { - committer.abortTask(taskContext) - } - logError(s"Task attempt $taskContext aborted.") - } - - private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { - jobID = jobId - splitID = splitId - attemptID = attemptId - - jID = new SerializableWritable[JobID](SparkHadoopWriterUtils.createJobID(now, jobId)) - taID = new SerializableWritable[TaskAttemptID]( - new TaskAttemptID(new TaskID(jID.value, TaskType.MAP, splitID), attemptID)) - } - - private def setConfParams() { - conf.value.set("mapred.job.id", jID.value.toString) - conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) - conf.value.set("mapred.task.id", taID.value.toString) - conf.value.setBoolean("mapred.task.is.map", true) - conf.value.setInt("mapred.task.partition", splitID) - } - - def newSerializer(tableDesc: TableDesc): Serializer = { - val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] - serializer.initialize(null, tableDesc.getProperties) - serializer - } - - protected def prepareForWrite() = { - val serializer = newSerializer(fileSinkConf.getTableInfo) - val standardOI = ObjectInspectorUtils - .getStandardObjectInspector( - fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, - ObjectInspectorCopyOption.JAVA) - .asInstanceOf[StructObjectInspector] - - val fieldOIs = standardOI.getAllStructFieldRefs.asScala.map(_.getFieldObjectInspector).toArray - val dataTypes = inputSchema.map(_.dataType) - val wrappers = fieldOIs.zip(dataTypes).map { case (f, dt) => wrapperFor(f, dt) } - val outputData = new Array[Any](fieldOIs.length) - (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) - } - - // this function is executed on executor side - def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = { - val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() - executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) - - iterator.foreach { row => - var i = 0 - while (i < fieldOIs.length) { - outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) - i += 1 - } - writer.write(serializer.serialize(outputData, standardOI)) - } - - close() - } -} - -private[hive] object SparkHiveWriterContainer { - def createPathFromString(path: String, conf: JobConf): Path = { - if (path == null) { - throw new IllegalArgumentException("Output path is null") - } - val outputPath = new Path(path) - val fs = outputPath.getFileSystem(conf) - if (outputPath == null || fs == null) { - throw new IllegalArgumentException("Incorrectly formatted output path") - } - outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - } -} - -private[spark] object SparkHiveDynamicPartitionWriterContainer { - val SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = "mapreduce.fileoutputcommitter.marksuccessfuljobs" -} - -private[spark] class SparkHiveDynamicPartitionWriterContainer( - jobConf: JobConf, - fileSinkConf: FileSinkDesc, - dynamicPartColNames: Array[String], - inputSchema: Seq[Attribute]) - extends SparkHiveWriterContainer(jobConf, fileSinkConf, inputSchema) { - - import SparkHiveDynamicPartitionWriterContainer._ - - private val defaultPartName = jobConf.get( - ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultStrVal) - - override protected def initWriters(): Unit = { - // do nothing - } - - override def close(): Unit = { - // do nothing - } - - override def commitJob(): Unit = { - // This is a hack to avoid writing _SUCCESS mark file. In lower versions of Hadoop (e.g. 1.0.4), - // semantics of FileSystem.globStatus() is different from higher versions (e.g. 2.4.1) and will - // include _SUCCESS file when glob'ing for dynamic partition data files. - // - // Better solution is to add a step similar to what Hive FileSinkOperator.jobCloseOp does: - // calling something like Utilities.mvFileToFinalPath to cleanup the output directory and then - // load it with loadDynamicPartitions/loadPartition/loadTable. - val oldMarker = conf.value.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true) - conf.value.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, false) - super.commitJob() - conf.value.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, oldMarker) - } - - // this function is executed on executor side - override def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = { - val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() - executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) - - val partitionOutput = inputSchema.takeRight(dynamicPartColNames.length) - val dataOutput = inputSchema.take(fieldOIs.length) - // Returns the partition key given an input row - val getPartitionKey = UnsafeProjection.create(partitionOutput, inputSchema) - // Returns the data columns to be written given an input row - val getOutputRow = UnsafeProjection.create(dataOutput, inputSchema) - - val fun: AnyRef = (pathString: String) => FileUtils.escapePathName(pathString, defaultPartName) - // Expressions that given a partition key build a string like: col1=val/col2=val/... - val partitionStringExpression = partitionOutput.zipWithIndex.flatMap { case (c, i) => - val escaped = - ScalaUDF(fun, StringType, Seq(Cast(c, StringType)), Seq(StringType)) - val str = If(IsNull(c), Literal(defaultPartName), escaped) - val partitionName = Literal(dynamicPartColNames(i) + "=") :: str :: Nil - if (i == 0) partitionName else Literal(Path.SEPARATOR_CHAR.toString) :: partitionName - } - - // Returns the partition path given a partition key. - val getPartitionString = - UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionOutput) - - // If anything below fails, we should abort the task. - try { - val sorter: UnsafeKVExternalSorter = new UnsafeKVExternalSorter( - StructType.fromAttributes(partitionOutput), - StructType.fromAttributes(dataOutput), - SparkEnv.get.blockManager, - SparkEnv.get.serializerManager, - TaskContext.get().taskMemoryManager().pageSizeBytes, - SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", - UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD)) - - while (iterator.hasNext) { - val inputRow = iterator.next() - val currentKey = getPartitionKey(inputRow) - sorter.insertKV(currentKey, getOutputRow(inputRow)) - } - - logInfo(s"Sorting complete. Writing out partition files one at a time.") - val sortedIterator = sorter.sortedIterator() - var currentKey: InternalRow = null - var currentWriter: FileSinkOperator.RecordWriter = null - try { - while (sortedIterator.next()) { - if (currentKey != sortedIterator.getKey) { - if (currentWriter != null) { - currentWriter.close(false) - } - currentKey = sortedIterator.getKey.copy() - logDebug(s"Writing partition: $currentKey") - currentWriter = newOutputWriter(currentKey) - } - - var i = 0 - while (i < fieldOIs.length) { - outputData(i) = if (sortedIterator.getValue.isNullAt(i)) { - null - } else { - wrappers(i)(sortedIterator.getValue.get(i, dataTypes(i))) - } - i += 1 - } - currentWriter.write(serializer.serialize(outputData, standardOI)) - } - } finally { - if (currentWriter != null) { - currentWriter.close(false) - } - } - commit() - } catch { - case cause: Throwable => - logError("Aborting task.", cause) - abortTask() - throw new SparkException("Task failed while writing rows.", cause) - } - /** Open and returns a new OutputWriter given a partition key. */ - def newOutputWriter(key: InternalRow): FileSinkOperator.RecordWriter = { - val partitionPath = getPartitionString(key).getString(0) - val newFileSinkDesc = new FileSinkDesc( - fileSinkConf.getDirName + partitionPath, - fileSinkConf.getTableInfo, - fileSinkConf.getCompressed) - newFileSinkDesc.setCompressCodec(fileSinkConf.getCompressCodec) - newFileSinkDesc.setCompressType(fileSinkConf.getCompressType) - - // use the path like ${hive_tmp}/_temporary/${attemptId}/ - // to avoid write to the same file when `spark.speculation=true` - val path = FileOutputFormat.getTaskOutputPath( - conf.value, - partitionPath.stripPrefix("/") + "/" + getOutputName) - - HiveFileFormatUtils.getHiveRecordWriter( - conf.value, - fileSinkConf.getTableInfo, - conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], - newFileSinkDesc, - path, - Reporter.NULL) - } - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index dcb8e498a406a..3267c237c865a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -431,7 +431,7 @@ private[hive] class TestHiveSparkSession( sharedState.cacheManager.clearCache() loadedTables.clear() sessionState.catalog.clearTempTables() - sessionState.catalog.invalidateCache() + sessionState.catalog.tableRelationCache.invalidateAll() sessionState.metadataHive.reset() diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index 072bb25d30a87..a48ae9fc5edd8 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -61,5 +61,5 @@ log4j.additivity.org.apache.hadoop.hive.ql.io.RCFile=false log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR # Parquet related logging -log4j.logger.org.apache.parquet=ERROR -log4j.logger.parquet=ERROR +log4j.logger.org.apache.parquet.CorruptStatistics=ERROR +log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql index 3de4f8a059965..adddc41024aca 100644 --- a/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql +++ b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql @@ -5,4 +5,4 @@ FROM (SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11) ORDER BY subq.key1, z.value -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key1`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_7` AS `gen_attr_6`, `gen_attr_9` AS `gen_attr_8`, `gen_attr_11` AS `gen_attr_10` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_7` FROM `default`.`src1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`src`) AS gen_subquery_1 ON (`gen_attr_5` = `gen_attr_9`)) AS subq INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1`, `ds` AS `gen_attr_3`, `hr` AS `gen_attr_4` FROM `default`.`srcpart`) AS gen_subquery_2 ON (((`gen_attr_0` = `gen_attr_2`) AND (`gen_attr_3` = '2008-04-08')) AND (CAST(`gen_attr_4` AS DOUBLE) = CAST(11 AS DOUBLE))) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_3 +SELECT `gen_attr_0` AS `key1`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_7` AS `gen_attr_6`, `gen_attr_9` AS `gen_attr_8`, `gen_attr_11` AS `gen_attr_10` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_7` FROM `default`.`src1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`src`) AS gen_subquery_1 ON (`gen_attr_5` = `gen_attr_9`)) AS subq INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1`, `ds` AS `gen_attr_3`, `hr` AS `gen_attr_4` FROM `default`.`srcpart`) AS gen_subquery_2 ON (((`gen_attr_0` = `gen_attr_2`) AND (`gen_attr_3` = '2008-04-08')) AND (CAST(`gen_attr_4` AS INT) = 11)) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql index 25882147463b9..55afb869d7cf7 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql @@ -5,4 +5,4 @@ group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key) order by key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (`gen_attr_2` IN (SELECT `gen_attr_5` AS `_c0` FROM (SELECT `gen_attr_3` AS `gen_attr_5` FROM (SELECT count(1) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_7` FROM `default`.`src`) AS gen_subquery_3 WHERE (CAST(`gen_attr_6` AS DOUBLE) = CAST('90' AS DOUBLE)) GROUP BY `gen_attr_6`) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS src +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (`gen_attr_2` IN (SELECT `gen_attr_5` AS `_c0` FROM (SELECT `gen_attr_3` AS `gen_attr_5` FROM (SELECT count(1) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_7` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_6` = CAST('90' AS INT)) GROUP BY `gen_attr_6`) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS src diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala index 27ea167b9050c..df9390aec7f7e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala @@ -93,6 +93,7 @@ class ExpressionToSQLSuite extends SQLBuilderTest with SQLTestUtils { checkSqlGeneration("SELECT array(1,2,3)") checkSqlGeneration("SELECT coalesce(null, 1, 2)") checkSqlGeneration("SELECT explode(array(1,2,3))") + checkSqlGeneration("SELECT explode_outer(array())") checkSqlGeneration("SELECT greatest(1,null,3)") checkSqlGeneration("SELECT if(1==2, 'yes', 'no')") checkSqlGeneration("SELECT isnan(15), isnan('invalid')") @@ -102,6 +103,8 @@ class ExpressionToSQLSuite extends SQLBuilderTest with SQLTestUtils { checkSqlGeneration("SELECT map(1, 'a', 2, 'b')") checkSqlGeneration("SELECT named_struct('c1',1,'c2',2,'c3',3)") checkSqlGeneration("SELECT nanvl(a, 5), nanvl(b, 10), nanvl(d, c) from t2") + checkSqlGeneration("SELECT posexplode_outer(array())") + checkSqlGeneration("SELECT inline_outer(array(struct('a', 1)))") checkSqlGeneration("SELECT rand(1)") checkSqlGeneration("SELECT randn(3)") checkSqlGeneration("SELECT struct(1,2,3)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index b67e5f6fe57a1..6f43b83607184 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -75,7 +75,8 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle assert(desc.comment == Some("This is the staging page view table")) // TODO will be SQLText assert(desc.viewText.isEmpty) - assert(desc.viewOriginalText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) assert(desc.partitionColumnNames.isEmpty) assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) @@ -106,7 +107,8 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle // TODO will be SQLText assert(desc.comment == Some("This is the staging page view table")) assert(desc.viewText.isEmpty) - assert(desc.viewOriginalText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) assert(desc.partitionColumnNames.isEmpty) assert(desc.storage.properties == Map()) assert(desc.storage.inputFormat == Some("parquet.hive.DeprecatedParquetInputFormat")) @@ -125,7 +127,8 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle assert(desc.storage.locationUri == None) assert(desc.schema.isEmpty) assert(desc.viewText == None) // TODO will be SQLText - assert(desc.viewOriginalText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) assert(desc.storage.properties == Map()) assert(desc.storage.inputFormat == Some("org.apache.hadoop.mapred.TextInputFormat")) assert(desc.storage.outputFormat == @@ -161,7 +164,8 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle assert(desc.storage.locationUri == None) assert(desc.schema.isEmpty) assert(desc.viewText == None) // TODO will be SQLText - assert(desc.viewOriginalText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) assert(desc.storage.properties == Map(("serde_p1" -> "p1"), ("serde_p2" -> "p2"))) assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) @@ -304,7 +308,8 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle assert(desc.partitionColumnNames.isEmpty) assert(desc.bucketSpec.isEmpty) assert(desc.viewText.isEmpty) - assert(desc.viewOriginalText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) assert(desc.storage.locationUri.isEmpty) assert(desc.storage.inputFormat == Some("org.apache.hadoop.mapred.TextInputFormat")) @@ -462,7 +467,8 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle assert(desc.partitionColumnNames == Seq("month")) assert(desc.bucketSpec.isEmpty) assert(desc.viewText.isEmpty) - assert(desc.viewOriginalText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) assert(desc.storage.locationUri == Some("/path/to/mercury")) assert(desc.storage.inputFormat == Some("winput")) assert(desc.storage.outputFormat == Some("wowput")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 2f02bb5d3b649..a60c210b04c8a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -41,6 +41,7 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { override val tableInputFormat: String = "org.apache.hadoop.mapred.SequenceFileInputFormat" override val tableOutputFormat: String = "org.apache.hadoop.mapred.SequenceFileOutputFormat" override def newEmptyCatalog(): ExternalCatalog = externalCatalog + override val defaultProvider: String = "hive" } protected override def resetState(): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 081f6f6d8263f..c262095df65b4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -419,12 +419,6 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql(s"CREATE TABLE $tableName STORED AS SEQUENCEFILE AS SELECT 1 AS key, 'abc' AS value") val df = sql(s"SELECT key, value FROM $tableName") - val e = intercept[AnalysisException] { - df.write.mode(SaveMode.Append).saveAsTable(tableName) - }.getMessage - assert(e.contains("Saving data in the Hive serde table default.tab1 is not supported " + - "yet. Please use the insertInto() API as an alternative.")) - df.write.insertInto(tableName) checkAnswer( sql(s"SELECT * FROM $tableName"), @@ -1167,8 +1161,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("create a temp view using hive") { val tableName = "tab1" - withTable (tableName) { - val e = intercept[ClassNotFoundException] { + withTable(tableName) { + val e = intercept[AnalysisException] { sql( s""" |CREATE TEMPORARY VIEW $tableName @@ -1176,7 +1170,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv |USING hive """.stripMargin) }.getMessage - assert(e.contains("Failed to find data source: hive")) + assert(e.contains("Hive data source can only be used with tables, you can't use it with " + + "CREATE TEMP VIEW USING")) } } @@ -1322,4 +1317,26 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sparkSession.sparkContext.conf.set(DEBUG_MODE, previousValue) } } + + test("SPARK-18464: support old table which doesn't store schema in table properties") { + withTable("old") { + withTempPath { path => + Seq(1 -> "a").toDF("i", "j").write.parquet(path.getAbsolutePath) + val tableDesc = CatalogTable( + identifier = TableIdentifier("old", Some("default")), + tableType = CatalogTableType.EXTERNAL, + storage = CatalogStorageFormat.empty.copy( + properties = Map("path" -> path.getAbsolutePath) + ), + schema = new StructType(), + properties = Map( + HiveExternalCatalog.DATASOURCE_PROVIDER -> "parquet")) + hiveClient.createTable(tableDesc, ignoreIfExists = false) + + checkAnswer(spark.table("old"), Row(1, "a")) + + checkAnswer(sql("DESC old"), Row("i", "int", null) :: Row("j", "string", null) :: Nil) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala index cc26b32184975..4bfab0f9cfbfb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala @@ -346,23 +346,10 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing table.copy( createTime = 0L, lastAccessTime = 0L, - properties = table.properties.filterKeys(!nondeterministicProps.contains(_)), - // View texts are checked separately - viewOriginalText = None, - viewText = None + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)) ) } - // Normalizes attributes auto-generated by Spark SQL for views - def normalizeGeneratedAttributes(str: String): String = { - str.replaceAll("gen_attr_[0-9]+", "gen_attr_0") - } - - // We use expanded canonical view text as original view text of the new table - assertResult(expected.viewText.map(normalizeGeneratedAttributes)) { - actual.viewOriginalText.map(normalizeGeneratedAttributes) - } - assert(normalize(actual) == normalize(expected)) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 0053aa1642ce2..e2fcd2fd41fa1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -62,7 +62,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto spark.conf.set(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS.key, true) - val relation = spark.sessionState.catalog.lookupRelation(TableIdentifier("csv_table")) + val relation = spark.table("csv_table").queryExecution.analyzed.children.head .asInstanceOf[MetastoreRelation] val properties = relation.hiveQlTable.getParameters @@ -80,7 +80,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto test("analyze MetastoreRelations") { def queryTotalSize(tableName: String): BigInt = - spark.sessionState.catalog.lookupRelation(TableIdentifier(tableName)).stats(conf).sizeInBytes + spark.table(tableName).queryExecution.analyzed.stats(conf).sizeInBytes // Non-partitioned table sql("CREATE TABLE analyzeTable (key STRING, value STRING)").collect() @@ -451,7 +451,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") } // Table lookup will make the table cached. - catalog.lookupRelation(tableIndent) + spark.table(tableIndent) statsBeforeUpdate = catalog.getCachedDataSourceTable(tableIndent) .asInstanceOf[LogicalRelation].catalogTable.get.stats.get @@ -461,7 +461,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto } else { sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") } - catalog.lookupRelation(tableIndent) + spark.table(tableIndent) statsAfterUpdate = catalog.getCachedDataSourceTable(tableIndent) .asInstanceOf[LogicalRelation].catalogTable.get.stats.get } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 5cb8519d2a9af..28b5bfd5819c6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -565,8 +565,8 @@ class VersionsSuite extends SparkFunSuite with SQLTestUtils with TestHiveSinglet val filePaths = dir.map(_.getName).toList folders.flatMap(listFiles) ++: filePaths } - val expectedFiles = ".part-00000.crc" :: "part-00000" :: Nil - assert(listFiles(tmpDir).sorted == expectedFiles) + // expect 2 files left: `.part-00000-random-uuid.crc` and `part-00000-random-uuid` + assert(listFiles(tmpDir).length == 2) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 05a15166f815b..4772a264d6fd9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -26,6 +26,7 @@ import scala.util.control.NonFatal import org.scalatest.{BeforeAndAfterAll, GivenWhenThen} import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.SQLBuilder import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ @@ -441,23 +442,20 @@ abstract class HiveComparisonTest val executions = queryList.map(new TestHiveQueryExecution(_)) executions.foreach(_.toRdd) val tablesGenerated = queryList.zip(executions).flatMap { - // We should take executedPlan instead of sparkPlan, because in following codes we - // will run the collected plans. As we will do extra processing for sparkPlan such - // as adding exchange, collapsing codegen stages, etc., collecting sparkPlan here - // will cause some errors when running these plans later. - case (q, e) => e.executedPlan.collect { + case (q, e) => e.analyzed.collect { case i: InsertIntoHiveTable if tablesRead contains i.table.tableName => (q, e, i) } } tablesGenerated.map { case (hiveql, execution, insert) => + val rdd = Dataset.ofRows(TestHive.sparkSession, insert.query).queryExecution.toRdd s""" |=== Generated Table === |$hiveql |$execution |== Results == - |${insert.child.execute().collect().mkString("\n")} + |${rdd.collect().mkString("\n")} """.stripMargin }.mkString("\n") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index e3f1667249684..58be079d01410 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -22,6 +22,7 @@ import java.io.File import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach +import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTableType} @@ -37,7 +38,7 @@ import org.apache.spark.sql.types.StructType class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { - import spark.implicits._ + import testImplicits._ override def afterEach(): Unit = { try { @@ -221,8 +222,8 @@ class HiveDDLSuite sql( s""" |ALTER TABLE $tab ADD - |PARTITION (ds='2008-04-08', hr=11) LOCATION '$part1Path' - |PARTITION (ds='2008-04-08', hr=12) LOCATION '$part2Path' + |PARTITION (ds='2008-04-08', hr=11) LOCATION '${part1Path.toURI}' + |PARTITION (ds='2008-04-08', hr=12) LOCATION '${part2Path.toURI}' """.stripMargin) assert(dirSet.forall(dir => dir.listFiles == null || dir.listFiles.isEmpty)) @@ -247,6 +248,16 @@ class HiveDDLSuite } } + test("SPARK-19129: drop partition with a empty string will drop the whole table") { + val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name") + df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable") + val e = intercept[AnalysisException] { + spark.sql("alter table partitionedTable drop partition(partCol1='')") + }.getMessage + assert(e.contains("Partition spec is invalid. The spec ([partCol1=]) contains an empty " + + "partition column value")) + } + test("add/drop partitions - external table") { val catalog = spark.sessionState.catalog withTempDir { tmpDir => @@ -370,28 +381,30 @@ class HiveDDLSuite spark.range(10).write.saveAsTable(tabName) val viewName = "view1" withView(viewName) { - val catalog = spark.sessionState.catalog + def checkProperties(expected: Map[String, String]): Boolean = { + val properties = spark.sessionState.catalog.getTableMetadata(TableIdentifier(viewName)) + .properties + properties.filterNot { case (key, value) => + Seq("transient_lastDdlTime", CatalogTable.VIEW_DEFAULT_DATABASE).contains(key) || + key.startsWith(CatalogTable.VIEW_QUERY_OUTPUT_PREFIX) + } == expected + } sql(s"CREATE VIEW $viewName AS SELECT * FROM $tabName") - assert(catalog.getTableMetadata(TableIdentifier(viewName)) - .properties.filter(_._1 != "transient_lastDdlTime") == Map()) + checkProperties(Map()) sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") - assert(catalog.getTableMetadata(TableIdentifier(viewName)) - .properties.filter(_._1 != "transient_lastDdlTime") == Map("p" -> "an")) + checkProperties(Map("p" -> "an")) // no exception or message will be issued if we set it again sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") - assert(catalog.getTableMetadata(TableIdentifier(viewName)) - .properties.filter(_._1 != "transient_lastDdlTime") == Map("p" -> "an")) + checkProperties(Map("p" -> "an")) // the value will be updated if we set the same key to a different value sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'b')") - assert(catalog.getTableMetadata(TableIdentifier(viewName)) - .properties.filter(_._1 != "transient_lastDdlTime") == Map("p" -> "b")) + checkProperties(Map("p" -> "b")) sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") - assert(catalog.getTableMetadata(TableIdentifier(viewName)) - .properties.filter(_._1 != "transient_lastDdlTime") == Map()) + checkProperties(Map()) val message = intercept[AnalysisException] { sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") @@ -643,11 +656,9 @@ class HiveDDLSuite assert(sql("DESC FORMATTED view1").collect().containsSlice( Seq( Row("# View Information", "", ""), - Row("View Original Text:", "SELECT * FROM tbl", ""), - Row("View Expanded Text:", - "SELECT `gen_attr_0` AS `a` FROM (SELECT `gen_attr_0` FROM " + - "(SELECT `a` AS `gen_attr_0` FROM `default`.`tbl`) AS gen_subquery_0) AS tbl", - "") + Row("View Text:", "SELECT * FROM tbl", ""), + Row("View Default Database:", "default", ""), + Row("View Query Output Columns:", "[a]", "") ) )) } @@ -789,7 +800,7 @@ class HiveDDLSuite test("Create Cataloged Table As Select - Drop Table After Runtime Exception") { withTable("tab") { - intercept[RuntimeException] { + intercept[SparkException] { sql( """ |CREATE TABLE tab @@ -933,7 +944,9 @@ class HiveDDLSuite TableIdentifier(sourceViewName, Some("default"))) // The original source should be a VIEW with an empty path assert(sourceView.tableType == CatalogTableType.VIEW) - assert(sourceView.viewText.nonEmpty && sourceView.viewOriginalText.nonEmpty) + assert(sourceView.viewText.nonEmpty) + assert(sourceView.viewDefaultDatabase == Some("default")) + assert(sourceView.viewQueryColumnNames == Seq("a", "b", "c", "d")) val targetTable = spark.sessionState.catalog.getTableMetadata( TableIdentifier(targetTabName, Some("default"))) @@ -946,8 +959,12 @@ class HiveDDLSuite // The created table should be a MANAGED table with empty view text and original text. assert(targetTable.tableType == CatalogTableType.MANAGED, "the created table must be a Hive managed table") - assert(targetTable.viewText.isEmpty && targetTable.viewOriginalText.isEmpty, - "the view text and original text in the created table must be empty") + assert(targetTable.viewText.isEmpty, + "the view text in the created table must be empty") + assert(targetTable.viewDefaultDatabase.isEmpty, + "the view default database in the created table must be empty") + assert(targetTable.viewQueryColumnNames.isEmpty, + "the view query output columns in the created table must be empty") assert(targetTable.comment.isEmpty, "the comment in the created table must be empty") assert(targetTable.unsupportedFeatures.isEmpty, @@ -1252,7 +1269,7 @@ class HiveDDLSuite s""" |CREATE TABLE t(id int) USING hive |OPTIONS(fileFormat 'orc', compression 'Zlib') - |LOCATION '${path.getCanonicalPath}' + |LOCATION '${path.toURI}' """.stripMargin) val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) assert(DDLUtils.isHiveTable(table)) @@ -1263,7 +1280,7 @@ class HiveDDLSuite sql("INSERT INTO t SELECT 1") checkAnswer(spark.table("t"), Row(1)) // Check if this is compressed as ZLIB. - val maybeOrcFile = path.listFiles().find(_.getName.endsWith("part-00000")) + val maybeOrcFile = path.listFiles().find(!_.getName.endsWith(".crc")) assert(maybeOrcFile.isDefined) val orcFilePath = maybeOrcFile.get.toPath.toString val expectedCompressionKind = @@ -1309,7 +1326,7 @@ class HiveDDLSuite } test("create hive serde table with DataFrameWriter.saveAsTable") { - withTable("t", "t2") { + withTable("t", "t1") { Seq(1 -> "a").toDF("i", "j") .write.format("hive").option("fileFormat", "avro").saveAsTable("t") checkAnswer(spark.table("t"), Row(1, "a")) @@ -1340,17 +1357,8 @@ class HiveDDLSuite assert(table.storage.serde == Some("org.apache.hadoop.hive.serde2.avro.AvroSerDe")) - sql("INSERT INTO t SELECT 2, 'b'") - checkAnswer(spark.table("t"), Row(9, "x") :: Row(2, "b") :: Nil) - - val e = intercept[AnalysisException] { - Seq(1 -> "a").toDF("i", "j").write.format("hive").partitionBy("i").saveAsTable("t2") - } - assert(e.message.contains("A Create Table As Select (CTAS) statement is not allowed " + - "to create a partitioned table using Hive")) - val e2 = intercept[AnalysisException] { - Seq(1 -> "a").toDF("i", "j").write.format("hive").bucketBy(4, "i").saveAsTable("t2") + Seq(1 -> "a").toDF("i", "j").write.format("hive").bucketBy(4, "i").saveAsTable("t1") } assert(e2.message.contains("Creating bucketed Hive serde table is not supported yet")) @@ -1361,6 +1369,51 @@ class HiveDDLSuite } } + test("append data to hive serde table") { + withTable("t", "t1") { + Seq(1 -> "a").toDF("i", "j") + .write.format("hive").option("fileFormat", "avro").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, "a")) + + sql("INSERT INTO t SELECT 2, 'b'") + checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Nil) + + Seq(3 -> "c").toDF("i", "j") + .write.format("hive").mode("append").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Nil) + + Seq("c" -> 3).toDF("i", "j") + .write.format("hive").mode("append").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") + :: Row(null, "3") :: Nil) + + Seq(4 -> "d").toDF("i", "j").write.saveAsTable("t1") + + val e = intercept[AnalysisException] { + Seq(5 -> "e").toDF("i", "j") + .write.format("hive").mode("append").saveAsTable("t1") + } + assert(e.message.contains("The format of the existing table default.t1 is " + + "`ParquetFileFormat`. It doesn't match the specified format `HiveFileFormat`.")) + } + } + + test("create partitioned hive serde table as select") { + withTable("t", "t1") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + Seq(10 -> "y").toDF("i", "j").write.format("hive").partitionBy("i").saveAsTable("t") + checkAnswer(spark.table("t"), Row("y", 10) :: Nil) + + Seq((1, 2, 3)).toDF("i", "j", "k").write.mode("overwrite").format("hive") + .partitionBy("j", "k").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, 2, 3) :: Nil) + + spark.sql("create table t1 using hive partitioned by (i) as select 1 as i, 'a' as j") + checkAnswer(spark.table("t1"), Row("a", 1) :: Nil) + } + } + } + test("read/write files with hive data source is not allowed") { withTempDir { dir => val e = intercept[AnalysisException] { @@ -1372,6 +1425,54 @@ class HiveDDLSuite Seq(1 -> "a").toDF("i", "j").write.format("hive").save(dir.getAbsolutePath) } assert(e2.message.contains("Hive data source can only be used with tables")) + + val e3 = intercept[AnalysisException] { + spark.readStream.format("hive").load(dir.getAbsolutePath) + } + assert(e3.message.contains("Hive data source can only be used with tables")) + + val e4 = intercept[AnalysisException] { + spark.readStream.schema(new StructType()).parquet(dir.getAbsolutePath) + .writeStream.format("hive").start(dir.getAbsolutePath) + } + assert(e4.message.contains("Hive data source can only be used with tables")) + } + } + + test("partitioned table should always put partition columns at the end of table schema") { + def getTableColumns(tblName: String): Seq[String] = { + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tblName)).schema.map(_.name) + } + + withTable("t", "t1", "t2", "t3", "t4", "t5", "t6") { + sql("CREATE TABLE t(a int, b int, c int, d int) USING parquet PARTITIONED BY (d, b)") + assert(getTableColumns("t") == Seq("a", "c", "d", "b")) + + sql("CREATE TABLE t1 USING parquet PARTITIONED BY (d, b) AS SELECT 1 a, 1 b, 1 c, 1 d") + assert(getTableColumns("t1") == Seq("a", "c", "d", "b")) + + Seq((1, 1, 1, 1)).toDF("a", "b", "c", "d").write.partitionBy("d", "b").saveAsTable("t2") + assert(getTableColumns("t2") == Seq("a", "c", "d", "b")) + + withTempPath { path => + val dataPath = new File(new File(path, "d=1"), "b=1").getCanonicalPath + Seq(1 -> 1).toDF("a", "c").write.save(dataPath) + + sql(s"CREATE TABLE t3 USING parquet LOCATION '${path.getCanonicalPath}'") + assert(getTableColumns("t3") == Seq("a", "c", "d", "b")) + } + + sql("CREATE TABLE t4(a int, b int, c int, d int) USING hive PARTITIONED BY (d, b)") + assert(getTableColumns("t4") == Seq("a", "c", "d", "b")) + + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + sql("CREATE TABLE t5 USING hive PARTITIONED BY (d, b) AS SELECT 1 a, 1 b, 1 c, 1 d") + assert(getTableColumns("t5") == Seq("a", "c", "d", "b")) + + Seq((1, 1, 1, 1)).toDF("a", "b", "c", "d").write.format("hive") + .partitionBy("d", "b").saveAsTable("t6") + assert(getTableColumns("t6") == Seq("a", "c", "d", "b")) + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2ae66d1b2f8a6..dd278f683a3cd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -388,6 +388,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd } } + // Some tests suing script transformation are skipped as it requires `/bin/bash` which + // can be missing or differently located. createQueryTest("transform", "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src", skip = !TestUtils.testCommandAvailable("/bin/bash")) @@ -461,7 +463,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src; - """.stripMargin.replaceAll(System.lineSeparator(), " ")) + """.stripMargin.replaceAll(System.lineSeparator(), " "), + skip = !TestUtils.testCommandAvailable("/bin/bash")) createQueryTest("transform with SerDe4", """ @@ -470,7 +473,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES |('serialization.last.column.takes.rest'='true') FROM src; - """.stripMargin.replaceAll(System.lineSeparator(), " ")) + """.stripMargin.replaceAll(System.lineSeparator(), " "), + skip = !TestUtils.testCommandAvailable("/bin/bash")) createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") @@ -1043,8 +1047,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd assertResult(1, "Duplicated project detected\n" + analyzedPlan) { analyzedPlan.collect { - case _: Project => () - }.size + case i: InsertIntoHiveTable => i.query.collect { case p: Project => () }.size + }.sum } } @@ -1062,8 +1066,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd assertResult(2, "Duplicated project detected\n" + analyzedPlan) { analyzedPlan.collect { - case _: Project => () - }.size + case i: InsertIntoHiveTable => i.query.collect { case p: Project => () }.size + }.sum } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index b8af0b39c8392..ec620c2403e3c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -34,7 +34,7 @@ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") """.stripMargin) - sql(s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales") + sql(s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt").toURI}' INTO TABLE sales") } // table sales is not a cache table, and will be clear after reset diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 953e29127fb88..20f30e48aba44 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry, NoSuchPartitionException} import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveUtils, MetastoreRelation} @@ -513,8 +514,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { isDataSourceTable: Boolean, format: String, userSpecifiedLocation: Option[String] = None): Unit = { - val relation = EliminateSubqueryAliases( - sessionState.catalog.lookupRelation(TableIdentifier(tableName))) + var relation: LogicalPlan = null + withSQLConf( + HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false", + HiveUtils.CONVERT_METASTORE_ORC.key -> "false") { + relation = EliminateSubqueryAliases(spark.table(tableName).queryExecution.analyzed) + } val catalogTable = sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) relation match { @@ -1021,13 +1026,11 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // generates an invalid query plan. val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) read.json(rdd).createOrReplaceTempView("data") - val originalConf = sessionState.conf.convertCTAS - setConf(SQLConf.CONVERT_CTAS, false) - try { + withSQLConf(SQLConf.CONVERT_CTAS.key -> "false") { sql("CREATE TABLE explodeTest (key bigInt)") table("explodeTest").queryExecution.analyzed match { - case metastoreRelation: MetastoreRelation => // OK + case SubqueryAlias(_, r: MetastoreRelation, _) => // OK case _ => fail("To correctly test the fix of SPARK-5875, explodeTest should be a MetastoreRelation") } @@ -1040,8 +1043,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql("DROP TABLE explodeTest") dropTempTable("data") - } finally { - setConf(SQLConf.CONVERT_CTAS, originalConf) } } @@ -1460,6 +1461,23 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { }) } + test("run sql directly on files - hive") { + withTempPath(f => { + spark.range(100).toDF.write.parquet(f.getCanonicalPath) + + var e = intercept[AnalysisException] { + sql(s"select id from hive.`${f.getCanonicalPath}`") + } + assert(e.message.contains("Unsupported data source type for direct query on files: hive")) + + // data source type is case insensitive + e = intercept[AnalysisException] { + sql(s"select id from HIVE.`${f.getCanonicalPath}`") + } + assert(e.message.contains("Unsupported data source type for direct query on files: HIVE")) + }) + } + test("SPARK-8976 Wrong Result for Rollup #1") { checkAnswer(sql( "SELECT count(*) AS cnt, key % 5, grouping_id() FROM src GROUP BY key%5 WITH ROLLUP"), @@ -2013,4 +2031,17 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ) } } + + test("SPARK-19292: filter with partition columns should be case-insensitive on Hive tables") { + withTable("tbl") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + sql("CREATE TABLE tbl(i int, j int) USING hive PARTITIONED BY (j)") + sql("INSERT INTO tbl PARTITION(j=10) SELECT 1") + checkAnswer(spark.table("tbl"), Row(1, 10)) + + checkAnswer(sql("SELECT i, j FROM tbl WHERE J=10"), Row(1, 10)) + checkAnswer(spark.table("tbl").filter($"J" === 10), Row(1, 10)) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index 9bc078dbb0228..958001e8fdb2c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -222,13 +222,15 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("correctly parse CREATE VIEW statement") { - sql( - """CREATE VIEW IF NOT EXISTS - |default.testView (c1 COMMENT 'blabla', c2 COMMENT 'blabla') - |TBLPROPERTIES ('a' = 'b') - |AS SELECT * FROM jt""".stripMargin) - checkAnswer(sql("SELECT c1, c2 FROM testView ORDER BY c1"), (1 to 9).map(i => Row(i, i))) - sql("DROP VIEW testView") + withView("testView") { + sql( + """CREATE VIEW IF NOT EXISTS + |default.testView (c1 COMMENT 'blabla', c2 COMMENT 'blabla') + |TBLPROPERTIES ('a' = 'b') + |AS SELECT * FROM jt + |""".stripMargin) + checkAnswer(sql("SELECT c1, c2 FROM testView ORDER BY c1"), (1 to 9).map(i => Row(i, i))) + } } test("correctly parse CREATE TEMPORARY VIEW statement") { @@ -554,7 +556,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("x", "long").add("y", "long"), - viewOriginalText = Some("SELECT * FROM jt"), viewText = Some("SELECT * FROM jt"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default", CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2", @@ -565,7 +566,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("id", "long").add("id1", "long"), - viewOriginalText = Some("SELECT * FROM view1"), viewText = Some("SELECT * FROM view1"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> db, CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2", @@ -587,7 +587,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("n", "int"), - viewOriginalText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM w"), viewText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM w"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default", CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "1", @@ -604,7 +603,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("id", "long").add("id1", "long"), - viewOriginalText = Some("SELECT * FROM jt"), viewText = Some("SELECT * FROM jt"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default", CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2", @@ -632,7 +630,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("id", "long").add("id1", "long"), - viewOriginalText = Some("SELECT * FROM invalid_db.jt"), viewText = Some("SELECT * FROM invalid_db.jt"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default", CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2", @@ -647,7 +644,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("id", "long").add("id1", "long"), - viewOriginalText = Some("SELECT * FROM invalid_table"), viewText = Some("SELECT * FROM invalid_table"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default", CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2", @@ -662,7 +658,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("id", "long").add("id1", "long"), - viewOriginalText = Some("SELECT * FROM view2"), viewText = Some("SELECT * FROM view2"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default", CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2", @@ -685,7 +680,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("a", "int").add("b", "int"), - viewOriginalText = Some(s"SELECT * FROM hive_table"), viewText = Some("SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b` FROM (SELECT " + "`gen_attr_0`, `gen_attr_1` FROM (SELECT `a` AS `gen_attr_0`, `b` AS " + "`gen_attr_1` FROM hive_table) AS gen_subquery_0) AS hive_table") @@ -709,7 +703,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("x", "long").add("y", "long"), - viewOriginalText = Some("SELECT * FROM testTable"), viewText = Some("SELECT * FROM testTable"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default", CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2", @@ -742,7 +735,6 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = new StructType().add("id", "long").add("id1", "long"), - viewOriginalText = Some("SELECT * FROM testTable"), viewText = Some("SELECT * FROM testTable"), properties = Map(CatalogTable.VIEW_DEFAULT_DATABASE -> "default", CatalogTable.VIEW_QUERY_OUTPUT_NUM_COLUMNS -> "2", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index d3475a79a7fae..5318b4650b01f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -55,7 +55,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformation( + (child: SparkPlan) => new ScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), @@ -71,7 +71,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformation( + (child: SparkPlan) => new ScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), @@ -88,7 +88,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { val e = intercept[TestFailedException] { checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformation( + (child: SparkPlan) => new ScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), @@ -107,7 +107,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { val e = intercept[TestFailedException] { checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformation( + (child: SparkPlan) => new ScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), @@ -126,7 +126,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { val e = intercept[SparkException] { val plan = - new ScriptTransformation( + new ScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "some_non_existent_command", output = Seq(AttributeReference("a", StringType)()), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index aa4a150a4b807..e9239ea56f1fb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -612,7 +612,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { test("Explicitly added partitions should be readable after load") { withTable("test_added_partitions") { withTempDir { src => - val newPartitionDir = src.getCanonicalPath + val newPartitionDir = src.toURI.toString spark.range(2).selectExpr("cast(id as string)").toDF("a").write .mode("overwrite") .parquet(newPartitionDir) @@ -645,7 +645,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { test("Non-partitioned table readable after load") { withTable("tab") { withTempDir { src => - val newPartitionDir = src.getCanonicalPath + val newPartitionDir = src.toURI.toString spark.range(2).selectExpr("cast(id as string)").toDF("a").write .mode("overwrite") .parquet(newPartitionDir) diff --git a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js b/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js index f82323a1cdd94..d004f34ab186c 100644 --- a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js +++ b/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js @@ -169,7 +169,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { .style("cursor", "pointer") .attr("cx", function(d) { return x(d.x); }) .attr("cy", function(d) { return y(d.y); }) - .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "0";}) + .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "3";}) .on('mouseover', function(d) { var tip = formatYValue(d.y) + " " + unitY + " at " + timeFormat[d.x]; showBootstrapTooltip(d3.select(this).node(), tip); @@ -187,7 +187,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { .attr("stroke", function(d) { return isFailedBatch(d.x) ? "red" : "white";}) .attr("fill", function(d) { return isFailedBatch(d.x) ? "red" : "white";}) .attr("opacity", function(d) { return isFailedBatch(d.x) ? "1" : "0";}) - .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "0";}); + .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "3";}); }) .on("click", function(d) { if (lastTimeout != null) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index 54d736ee5101b..dce2028b48878 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -31,12 +31,15 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { private val inputStreams = new ArrayBuffer[InputDStream[_]]() private val outputStreams = new ArrayBuffer[DStream[_]]() + @volatile private var inputStreamNameAndID: Seq[(String, Int)] = Nil + var rememberDuration: Duration = null var checkpointInProgress = false var zeroTime: Time = null var startTime: Time = null var batchDuration: Duration = null + @volatile private var numReceivers: Int = 0 def start(time: Time) { this.synchronized { @@ -45,7 +48,9 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { startTime = time outputStreams.foreach(_.initialize(zeroTime)) outputStreams.foreach(_.remember(rememberDuration)) - outputStreams.foreach(_.validateAtStart) + outputStreams.foreach(_.validateAtStart()) + numReceivers = inputStreams.count(_.isInstanceOf[ReceiverInputDStream[_]]) + inputStreamNameAndID = inputStreams.map(is => (is.name, is.id)) inputStreams.par.foreach(_.start()) } } @@ -106,9 +111,9 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { .toArray } - def getInputStreamName(streamId: Int): Option[String] = synchronized { - inputStreams.find(_.id == streamId).map(_.name) - } + def getNumReceivers: Int = numReceivers + + def getInputStreamNameAndID: Seq[(String, Int)] = inputStreamNameAndID def generateJobs(time: Time): Seq[Job] = { logDebug("Generating jobs for time " + time) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index b7d114bc16d48..2fa3bf7d5230b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -201,18 +201,20 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { listenerBus.post(StreamingListenerOutputOperationCompleted(job.toOutputOperationInfo)) logInfo("Finished job " + job.id + " from job set of time " + jobSet.time) if (jobSet.hasCompleted) { - jobSets.remove(jobSet.time) - jobGenerator.onBatchCompletion(jobSet.time) - logInfo("Total delay: %.3f s for time %s (execution: %.3f s)".format( - jobSet.totalDelay / 1000.0, jobSet.time.toString, - jobSet.processingDelay / 1000.0 - )) listenerBus.post(StreamingListenerBatchCompleted(jobSet.toBatchInfo)) } job.result match { case Failure(e) => reportError("Error running job " + job, e) case _ => + if (jobSet.hasCompleted) { + jobSets.remove(jobSet.time) + jobGenerator.onBatchCompletion(jobSet.time) + logInfo("Total delay: %.3f s for time %s (execution: %.3f s)".format( + jobSet.totalDelay / 1000.0, jobSet.time.toString, + jobSet.processingDelay / 1000.0 + )) + } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 95f582106c71f..ed4c1e484efd2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -169,7 +169,7 @@ private[spark] class StreamingJobProgressListener(ssc: StreamingContext) } def numInactiveReceivers: Int = { - ssc.graph.getReceiverInputStreams().length - numActiveReceivers + ssc.graph.getNumReceivers - numActiveReceivers } def numTotalCompletedBatches: Long = synchronized { @@ -197,17 +197,17 @@ private[spark] class StreamingJobProgressListener(ssc: StreamingContext) } def retainedCompletedBatches: Seq[BatchUIData] = synchronized { - completedBatchUIData.toSeq + completedBatchUIData.toIndexedSeq } def streamName(streamId: Int): Option[String] = { - ssc.graph.getInputStreamName(streamId) + ssc.graph.getInputStreamNameAndID.find(_._2 == streamId).map(_._1) } /** * Return all InputDStream Ids */ - def streamIds: Seq[Int] = ssc.graph.getInputStreams().map(_.id) + def streamIds: Seq[Int] = ssc.graph.getInputStreamNameAndID.map(_._2) /** * Return all of the record rates for each InputDStream in each batch. The key of the return value
Property NameMeaning