diff --git a/.asf.yaml b/.asf.yaml new file mode 100644 index 0000000000000..c7267ca5f346a --- /dev/null +++ b/.asf.yaml @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# https://cwiki.apache.org/confluence/display/INFRA/.asf.yaml+features+for+git+repositories +--- +github: + description: "Apache Spark - A unified analytics engine for large-scale data processing" + homepage: https://spark.apache.org/ + labels: + - python + - scala + - r + - java + - big-data + - jdbc + - sql + - spark diff --git a/.github/autolabeler.yml b/.github/autolabeler.yml new file mode 100644 index 0000000000000..bd4ad33c65afa --- /dev/null +++ b/.github/autolabeler.yml @@ -0,0 +1,129 @@ +# +# 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 +# +# https://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. +# + +# Bot page: https://github.com/apps/probot-autolabeler +# The matching patterns follow the .gitignore spec. +# See: https://git-scm.com/docs/gitignore#_pattern_format +# Also, note that the plugin uses 'ignore' package. See also +# https://github.com/kaelzhang/node-ignore +INFRA: + - ".github/" + - "appveyor.yml" + - "/tools/" + - "/dev/create-release/" + - ".asf.yaml" + - ".gitattributes" + - ".gitignore" + - "/dev/github_jira_sync.py" + - "/dev/merge_spark_pr.py" + - "/dev/run-tests-jenkins*" +BUILD: + - "/dev/" + - "!/dev/github_jira_sync.py" + - "!/dev/merge_spark_pr.py" + - "!/dev/run-tests-jenkins*" + - "/build/" + - "/project/" + - "/assembly/" + - "*pom.xml" + - "/bin/docker-image-tool.sh" + - "/bin/find-spark-home*" + - "scalastyle-config.xml" +DOCS: + - "docs/" + - "/README.md" + - "/CONTRIBUTING.md" +EXAMPLES: + - "examples/" + - "/bin/run-example*" +CORE: + - "/core/" + - "/common/kvstore/" + - "/common/network-common/" + - "/common/network-shuffle/" + - "/python/pyspark/*.py" + - "/python/pyspark/tests/*.py" +SPARK SUBMIT: + - "/bin/spark-submit*" +SPARK SHELL: + - "/repl/" + - "/bin/spark-shell*" +SQL: + - "sql/" + - "/common/unsafe/" + - "!/python/pyspark/sql/avro/" + - "!/python/pyspark/sql/streaming.py" + - "!/python/pyspark/sql/tests/test_streaming.py" + - "/bin/spark-sql*" + - "/bin/beeline*" + - "/sbin/*thriftserver*.sh" + - "*SQL*.R" + - "DataFrame.R" + - "WindowSpec.R" + - "catalog.R" + - "column.R" + - "functions.R" + - "group.R" + - "schema.R" + - "types.R" +AVRO: + - "/external/avro/" + - "/python/pyspark/sql/avro/" +DSTREAM: + - "/streaming/" + - "/data/streaming/" + - "/external/flume*" + - "/external/kinesis*" + - "/external/kafka*" + - "/python/pyspark/streaming/" +GRAPHX: + - "/graphx/" + - "/data/graphx/" +ML: + - "ml/" + - "*mllib_*.R" +MLLIB: + - "spark/mllib/" + - "/mllib-local/" + - "/python/pyspark/mllib/" +STRUCTURED STREAMING: + - "sql/**/streaming/" + - "/external/kafka-0-10-sql/" + - "/python/pyspark/sql/streaming.py" + - "/python/pyspark/sql/tests/test_streaming.py" + - "*streaming.R" +PYTHON: + - "/bin/pyspark*" + - "python/" +R: + - "r/" + - "R/" + - "/bin/sparkR*" +YARN: + - "/resource-managers/yarn/" +MESOS: + - "/resource-managers/mesos/" + - "/sbin/*mesos*.sh" +KUBERNETES: + - "/resource-managers/kubernetes/" +WINDOWS: + - "*.cmd" + - "/R/pkg/tests/fulltests/test_Windows.R" +WEB UI: + - "ui/" +DEPLOY: + - "/sbin/" diff --git a/.gitignore b/.gitignore index 60a12e3d7b09b..198fdee39be95 100644 --- a/.gitignore +++ b/.gitignore @@ -18,8 +18,6 @@ .idea_modules/ .project .pydevproject -.python-version -.ruby-version .scala_dependencies .settings /lib/ diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 2f7b876f0ec33..53a0b7856567e 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -28,6 +28,7 @@ importFrom("utils", "download.file", "object.size", "packageVersion", "tail", "u # S3 methods exported export("sparkR.session") +export("sparkR.init") export("sparkR.session.stop") export("sparkR.stop") export("sparkR.conf") @@ -41,6 +42,9 @@ export("sparkR.callJStatic") export("install.spark") +export("sparkRSQL.init", + "sparkRHive.init") + # MLlib integration exportMethods("glm", "spark.glm", @@ -68,7 +72,10 @@ exportMethods("glm", "spark.freqItemsets", "spark.associationRules", "spark.findFrequentSequentialPatterns", - "spark.assignClusters") + "spark.assignClusters", + "spark.fmClassifier", + "spark.lm", + "spark.fmRegressor") # Job group lifecycle management methods export("setJobGroup", @@ -148,6 +155,7 @@ exportMethods("arrange", "printSchema", "randomSplit", "rbind", + "registerTempTable", "rename", "repartition", "repartitionByRange", @@ -345,6 +353,7 @@ exportMethods("%<=>%", "over", "overlay", "percent_rank", + "percentile_approx", "pmod", "posexplode", "posexplode_outer", @@ -430,8 +439,10 @@ export("as.DataFrame", "cacheTable", "clearCache", "createDataFrame", + "createExternalTable", "createTable", "currentDatabase", + "dropTempTable", "dropTempView", "listColumns", "listDatabases", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 593d3ca16220d..14d2076e88eff 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -521,6 +521,32 @@ setMethod("createOrReplaceTempView", invisible(callJMethod(x@sdf, "createOrReplaceTempView", viewName)) }) +#' (Deprecated) Register Temporary Table +#' +#' Registers a SparkDataFrame as a Temporary Table in the SparkSession +#' @param x A SparkDataFrame +#' @param tableName A character vector containing the name of the table +#' +#' @seealso \link{createOrReplaceTempView} +#' @rdname registerTempTable-deprecated +#' @name registerTempTable +#' @aliases registerTempTable,SparkDataFrame,character-method +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' registerTempTable(df, "json_df") +#' new_df <- sql("SELECT * FROM json_df") +#'} +#' @note registerTempTable since 1.4.0 +setMethod("registerTempTable", + signature(x = "SparkDataFrame", tableName = "character"), + function(x, tableName) { + .Deprecated("createOrReplaceTempView") + invisible(callJMethod(x@sdf, "createOrReplaceTempView", tableName)) + }) + #' insertInto #' #' Insert the contents of a SparkDataFrame into a table registered in the current SparkSession. diff --git a/R/pkg/R/catalog.R b/R/pkg/R/catalog.R index 7641f8a7a0432..275737f804bde 100644 --- a/R/pkg/R/catalog.R +++ b/R/pkg/R/catalog.R @@ -17,6 +17,35 @@ # catalog.R: SparkSession catalog functions +#' (Deprecated) Create an external table +#' +#' Creates an external table based on the dataset in a data source, +#' Returns a SparkDataFrame associated with the external table. +#' +#' The data source is specified by the \code{source} and a set of options(...). +#' If \code{source} is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param tableName a name of the table. +#' @param path the path of files to load. +#' @param source the name of external data source. +#' @param schema the schema of the data required for some data sources. +#' @param ... additional argument(s) passed to the method. +#' @return A SparkDataFrame. +#' @rdname createExternalTable-deprecated +#' @seealso \link{createTable} +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- createExternalTable("myjson", path="path/to/json", source="json", schema) +#' } +#' @name createExternalTable +#' @note createExternalTable since 1.4.0 +createExternalTable <- function(tableName, path = NULL, source = NULL, schema = NULL, ...) { + .Deprecated("createTable", old = "createExternalTable") + createTable(tableName, path, source, schema, ...) +} + #' Creates a table based on the dataset in a data source #' #' Creates a table based on the dataset in a data source. Returns a SparkDataFrame associated with @@ -130,6 +159,31 @@ clearCache <- function() { invisible(callJMethod(catalog, "clearCache")) } +#' (Deprecated) Drop Temporary Table +#' +#' Drops the temporary table with the given table name in the catalog. +#' If the table has been cached/persisted before, it's also unpersisted. +#' +#' @param tableName The name of the SparkSQL table to be dropped. +#' @seealso \link{dropTempView} +#' @rdname dropTempTable-deprecated +#' @examples +#' \dontrun{ +#' sparkR.session() +#' df <- read.df(path, "parquet") +#' createOrReplaceTempView(df, "table") +#' dropTempTable("table") +#' } +#' @name dropTempTable +#' @note dropTempTable since 1.4.0 +dropTempTable <- function(tableName) { + .Deprecated("dropTempView", old = "dropTempTable") + if (class(tableName) != "character") { + stop("tableName must be a string.") + } + dropTempView(tableName) +} + #' Drops the temporary view with the given view name in the catalog. #' #' Drops the temporary view with the given view name in the catalog. diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 79dd7bdc80eef..2baf3aaebf8bd 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -77,7 +77,13 @@ NULL #' days to be added to or subtracted from \code{y}. For class \code{character}, it is #' \itemize{ #' \item \code{date_format}: date format specification. -#' \item \code{from_utc_timestamp}, \code{to_utc_timestamp}: time zone to use. +#' \item \code{from_utc_timestamp}, \code{to_utc_timestamp}: A string detailing +#' the time zone ID that the input should be adjusted to. It should be in the format +#' of either region-based zone IDs or zone offsets. Region IDs must have the form +#' 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in the format +#' (+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported +#' as aliases of '+00:00'. Other short names are not recommended to use +#' because they can be ambiguous. #' \item \code{next_day}: day of the week string. #' } #' @param ... additional argument(s). @@ -1410,6 +1416,52 @@ setMethod("quarter", column(jc) }) +#' @details +#' \code{percentile_approx} Returns the approximate percentile value of +#' numeric column at the given percentage. +#' +#' @param percentage Numeric percentage at which percentile should be computed +#' All values should be between 0 and 1. +#' If length equals to 1 resulting column is of type double, +#' otherwise, array type of double. +#' @param accuracy A positive numeric literal (default: 10000) which +#' controls approximation accuracy at the cost of memory. +#' Higher value of accuracy yields better accuracy, 1.0/accuracy +#' is the relative error of the approximation. +#' +#' @rdname column_aggregate_functions +#' @aliases percentile_approx percentile_approx,Column-method +#' @note percentile_approx since 3.1.0 +setMethod("percentile_approx", + signature(x = "characterOrColumn", percentage = "numericOrColumn"), + function(x, percentage, accuracy = 10000) { + col <- if (class(x) == "Column") { + x@jc + } else { + column(x)@jc + } + + percentage <- if (class(percentage) == "Column") { + percentage@jc + } else if (length(percentage) > 1) { + do.call(create_array, lapply(percentage, lit))@jc + } else { + lit(percentage)@jc + } + + accuracy <- if (class(accuracy) == "Column") { + accuracy@jc + } else { + lit(as.integer(accuracy))@jc + } + + jc <- callJStatic( + "org.apache.spark.sql.functions", "percentile_approx", + col, percentage, accuracy + ) + column(jc) + }) + #' @details #' \code{reverse}: Returns a reversed string or an array with reverse order of elements. #' @@ -1833,7 +1885,7 @@ setMethod("radians", #' @details #' \code{to_date}: Converts the column into a DateType. You may optionally specify #' a format according to the rules in: -#' \url{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}. +#' \href{https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html}{Datetime Pattern} #' If the string cannot be parsed according to the specified format (or default), #' the value of the column will be null. #' By default, it follows casting rules to a DateType if the format is omitted @@ -1929,7 +1981,7 @@ setMethod("to_csv", signature(x = "Column"), #' @details #' \code{to_timestamp}: Converts the column into a TimestampType. You may optionally specify #' a format according to the rules in: -#' \url{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}. +#' \href{https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html}{Datetime Pattern} #' If the string cannot be parsed according to the specified format (or default), #' the value of the column will be null. #' By default, it follows casting rules to a TimestampType if the format is omitted @@ -2801,8 +2853,8 @@ setMethod("format_string", signature(format = "character", x = "Column"), #' \code{from_unixtime}: Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) #' to a string representing the timestamp of that moment in the current system time zone in the JVM #' in the given format. -#' See \href{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}{ -#' Customizing Formats} for available options. +#' See \href{https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html}{ +#' Datetime Pattern} for available options. #' #' @rdname column_datetime_functions #' @@ -2923,7 +2975,7 @@ setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), #' @details #' \code{rand}: Generates a random column with independent and identically distributed (i.i.d.) -#' samples from U[0.0, 1.0]. +#' samples uniformly distributed in [0.0, 1.0). #' Note: the function is non-deterministic in general case. #' #' @rdname column_nonaggregate_functions diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index a52ec7a4a27c1..839c00cf21aeb 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -528,6 +528,9 @@ setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) #' @rdname printSchema setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) +#' @rdname registerTempTable-deprecated +setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) + #' @rdname rename setGeneric("rename", function(x, ...) { standardGeneric("rename") }) @@ -1189,6 +1192,11 @@ setGeneric("overlay", function(x, replace, pos, ...) { standardGeneric("overlay" #' @name NULL setGeneric("percent_rank", function(x = "missing") { standardGeneric("percent_rank") }) +#' @rdname column_aggregate_functions +#' @name NULL +setGeneric("percentile_approx", + function(x, percentage, ...) { standardGeneric("percentile_approx") }) + #' @rdname column_math_functions #' @name NULL setGeneric("pmod", function(y, x) { standardGeneric("pmod") }) @@ -1471,6 +1479,14 @@ setGeneric("spark.als", function(data, ...) { standardGeneric("spark.als") }) setGeneric("spark.bisectingKmeans", function(data, formula, ...) { standardGeneric("spark.bisectingKmeans") }) +#' @rdname spark.fmClassifier +setGeneric("spark.fmClassifier", + function(data, formula, ...) { standardGeneric("spark.fmClassifier") }) + +#' @rdname spark.fmRegressor +setGeneric("spark.fmRegressor", + function(data, formula, ...) { standardGeneric("spark.fmRegressor") }) + #' @rdname spark.gaussianMixture setGeneric("spark.gaussianMixture", function(data, formula, ...) { standardGeneric("spark.gaussianMixture") }) @@ -1539,6 +1555,10 @@ setGeneric("spark.findFrequentSequentialPatterns", setGeneric("spark.assignClusters", function(data, ...) { standardGeneric("spark.assignClusters") }) +#' @rdname spark.lm +setGeneric("spark.lm", + function(data, formula, ...) { standardGeneric("spark.lm") }) + #' @param object a fitted ML model object. #' @param path the directory where the model is saved. #' @param ... additional argument(s) passed to the method. diff --git a/R/pkg/R/mllib_classification.R b/R/pkg/R/mllib_classification.R index 3ad824e1e634a..fc5ac9f06d8dc 100644 --- a/R/pkg/R/mllib_classification.R +++ b/R/pkg/R/mllib_classification.R @@ -42,6 +42,12 @@ setClass("MultilayerPerceptronClassificationModel", representation(jobj = "jobj" #' @note NaiveBayesModel since 2.0.0 setClass("NaiveBayesModel", representation(jobj = "jobj")) +#' S4 class that represents a FMClassificationModel +#' +#' @param jobj a Java object reference to the backing Scala FMClassifierWrapper +#' @note FMClassificationModel since 3.1.0 +setClass("FMClassificationModel", representation(jobj = "jobj")) + #' Linear SVM Model #' #' Fits a linear SVM model against a SparkDataFrame, similar to svm in e1071 package. @@ -649,3 +655,154 @@ setMethod("write.ml", signature(object = "NaiveBayesModel", path = "character"), function(object, path, overwrite = FALSE) { write_internal(object, path, overwrite) }) + +#' Factorization Machines Classification Model +#' +#' \code{spark.fmClassifier} fits a factorization classification 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. +#' Only categorical data is supported. +#' +#' @param data a \code{SparkDataFrame} of observations and labels for model fitting. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param factorSize dimensionality of the factors. +#' @param fitLinear whether to fit linear term. # TODO Can we express this with formula? +#' @param regParam the regularization parameter. +#' @param miniBatchFraction the mini-batch fraction parameter. +#' @param initStd the standard deviation of initial coefficients. +#' @param maxIter maximum iteration number. +#' @param stepSize stepSize parameter. +#' @param tol convergence tolerance of iterations. +#' @param solver solver parameter, supported options: "gd" (minibatch gradient descent) or "adamW". +#' @param thresholds in binary classification, in range [0, 1]. If the estimated probability of +#' class label 1 is > threshold, then predict 1, else 0. A high threshold +#' encourages the model to predict 0 more often; a low threshold encourages the +#' model to predict 1 more often. Note: Setting this with threshold p is +#' equivalent to setting thresholds c(1-p, p). +#' @param seed seed parameter for weights initialization. +#' @param handleInvalid How to handle invalid data (unseen labels or NULL values) in features and +#' label column of string type. +#' Supported options: "skip" (filter out rows with invalid data), +#' "error" (throw an error), "keep" (put invalid data in +#' a special additional bucket, at index numLabels). Default +#' is "error". +#' @param ... additional arguments passed to the method. +#' @return \code{spark.fmClassifier} returns a fitted Factorization Machines Classification Model. +#' @rdname spark.fmClassifier +#' @aliases spark.fmClassifier,SparkDataFrame,formula-method +#' @name spark.fmClassifier +#' @seealso \link{read.ml} +#' @examples +#' \dontrun{ +#' df <- read.df("data/mllib/sample_binary_classification_data.txt", source = "libsvm") +#' +#' # fit Factorization Machines Classification Model +#' model <- spark.fmClassifier( +#' df, label ~ features, +#' regParam = 0.01, maxIter = 10, fitLinear = TRUE +#' ) +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.fmClassifier since 3.1.0 +setMethod("spark.fmClassifier", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, factorSize = 8, fitLinear = TRUE, regParam = 0.0, + miniBatchFraction = 1.0, initStd = 0.01, maxIter = 100, stepSize=1.0, + tol = 1e-6, solver = c("adamW", "gd"), thresholds = NULL, seed = NULL, + handleInvalid = c("error", "keep", "skip")) { + + formula <- paste(deparse(formula), collapse = "") + + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + + if (!is.null(thresholds)) { + thresholds <- as.list(thresholds) + } + + solver <- match.arg(solver) + handleInvalid <- match.arg(handleInvalid) + + jobj <- callJStatic("org.apache.spark.ml.r.FMClassifierWrapper", + "fit", + data@sdf, + formula, + as.integer(factorSize), + as.logical(fitLinear), + as.numeric(regParam), + as.numeric(miniBatchFraction), + as.numeric(initStd), + as.integer(maxIter), + as.numeric(stepSize), + as.numeric(tol), + solver, + seed, + thresholds, + handleInvalid) + new("FMClassificationModel", jobj = jobj) + }) + +# Returns the summary of a FM Classification model produced by \code{spark.fmClassifier} + +#' @param object a FM Classification model fitted by \code{spark.fmClassifier}. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' @rdname spark.fmClassifier +#' @note summary(FMClassificationModel) since 3.1.0 +setMethod("summary", signature(object = "FMClassificationModel"), + function(object) { + jobj <- object@jobj + features <- callJMethod(jobj, "rFeatures") + coefficients <- callJMethod(jobj, "rCoefficients") + coefficients <- as.matrix(unlist(coefficients)) + colnames(coefficients) <- c("Estimate") + rownames(coefficients) <- unlist(features) + numClasses <- callJMethod(jobj, "numClasses") + numFeatures <- callJMethod(jobj, "numFeatures") + raw_factors <- unlist(callJMethod(jobj, "rFactors")) + factor_size <- callJMethod(jobj, "factorSize") + + list( + coefficients = coefficients, + factors = matrix(raw_factors, ncol = factor_size), + numClasses = numClasses, numFeatures = numFeatures, + factorSize = factor_size + ) + }) + +# Predicted values based on an FMClassificationModel model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns the predicted values based on a FM Classification model. +#' @rdname spark.fmClassifier +#' @aliases predict,FMClassificationModel,SparkDataFrame-method +#' @note predict(FMClassificationModel) since 3.1.0 +setMethod("predict", signature(object = "FMClassificationModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Save fitted FMClassificationModel 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.fmClassifier +#' @aliases write.ml,FMClassificationModel,character-method +#' @note write.ml(FMClassificationModel, character) since 3.1.0 +setMethod("write.ml", signature(object = "FMClassificationModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) diff --git a/R/pkg/R/mllib_regression.R b/R/pkg/R/mllib_regression.R index 4fabe9a0068fc..b2228a141689b 100644 --- a/R/pkg/R/mllib_regression.R +++ b/R/pkg/R/mllib_regression.R @@ -36,6 +36,18 @@ setClass("GeneralizedLinearRegressionModel", representation(jobj = "jobj")) #' @note IsotonicRegressionModel since 2.1.0 setClass("IsotonicRegressionModel", representation(jobj = "jobj")) +#' S4 class that represents a LinearRegressionModel +#' +#' @param jobj a Java object reference to the backing Scala LinearRegressionWrapper +#' @note LinearRegressionModel since 3.1.0 +setClass("LinearRegressionModel", representation(jobj = "jobj")) + +#' S4 class that represents a FMRegressionModel +#' +#' @param jobj a Java object reference to the backing Scala FMRegressorWrapper +#' @note FMRegressionModel since 3.1.0 +setClass("FMRegressionModel", representation(jobj = "jobj")) + #' Generalized Linear Models #' #' Fits generalized linear model against a SparkDataFrame. @@ -540,3 +552,314 @@ setMethod("write.ml", signature(object = "AFTSurvivalRegressionModel", path = "c function(object, path, overwrite = FALSE) { write_internal(object, path, overwrite) }) + +#' Linear Regression Model +#' +#' \code{spark.lm} fits a linear regression 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 \code{SparkDataFrame} of observations and labels for model fitting. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param maxIter maximum iteration number. +#' @param regParam the regularization parameter. +#' @param elasticNetParam the ElasticNet mixing parameter, in range [0, 1]. +#' For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty. +#' @param tol convergence tolerance of iterations. +#' @param standardization whether to standardize the training features before fitting the model. +#' @param weightCol weight column name. +#' @param aggregationDepth suggested depth for treeAggregate (>= 2). +#' @param loss the loss function to be optimized. Supported options: "squaredError" and "huber". +#' @param epsilon the shape parameter to control the amount of robustness. +#' @param solver The solver algorithm for optimization. +#' Supported options: "l-bfgs", "normal" and "auto". +#' @param stringIndexerOrderType how to order categories of a string feature column. This is used to +#' decide the base level of a string feature as the last category +#' after ordering is dropped when encoding strings. Supported options +#' are "frequencyDesc", "frequencyAsc", "alphabetDesc", and +#' "alphabetAsc". The default value is "frequencyDesc". When the +#' ordering is set to "alphabetDesc", this drops the same category +#' as R when encoding strings. +#' @param ... additional arguments passed to the method. +#' @return \code{spark.lm} returns a fitted Linear Regression Model. +#' @rdname spark.lm +#' @aliases spark.lm,SparkDataFrame,formula-method +#' @name spark.lm +#' @seealso \link{read.ml} +#' @examples +#' \dontrun{ +#' df <- read.df("data/mllib/sample_linear_regression_data.txt", source = "libsvm") +#' +#' # fit Linear Regression Model +#' model <- spark.lm(df, label ~ features, regParam = 0.01, maxIter = 1) +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.lm since 3.1.0 +setMethod("spark.lm", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, + maxIter = 100L, regParam = 0.0, elasticNetParam = 0.0, + tol = 1e-6, standardization = TRUE, + solver = c("auto", "l-bfgs", "normal"), + weightCol = NULL, aggregationDepth = 2L, + loss = c("squaredError", "huber"), epsilon = 1.35, + stringIndexerOrderType = c("frequencyDesc", "frequencyAsc", + "alphabetDesc", "alphabetAsc")) { + + + formula <- paste(deparse(formula), collapse = "") + + + solver <- match.arg(solver) + loss <- match.arg(loss) + stringIndexerOrderType <- match.arg(stringIndexerOrderType) + + + if (!is.null(weightCol) && weightCol == "") { + weightCol <- NULL + } else if (!is.null(weightCol)) { + weightCol <- as.character(weightCol) + } + + + jobj <- callJStatic("org.apache.spark.ml.r.LinearRegressionWrapper", + "fit", + data@sdf, + formula, + as.integer(maxIter), + as.numeric(regParam), + as.numeric(elasticNetParam), + as.numeric(tol), + as.logical(standardization), + solver, + weightCol, + as.integer(aggregationDepth), + loss, + as.numeric(epsilon), + stringIndexerOrderType) + new("LinearRegressionModel", jobj = jobj) + }) + + +# Returns the summary of a Linear Regression model produced by \code{spark.lm} + + +#' @param object a Linear Regression Model model fitted by \code{spark.lm}. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' +#' @rdname spark.lm +#' @note summary(LinearRegressionModel) since 3.1.0 +setMethod("summary", signature(object = "LinearRegressionModel"), + function(object) { + jobj <- object@jobj + features <- callJMethod(jobj, "rFeatures") + coefficients <- callJMethod(jobj, "rCoefficients") + coefficients <- as.matrix(unlist(coefficients)) + colnames(coefficients) <- c("Estimate") + rownames(coefficients) <- unlist(features) + numFeatures <- callJMethod(jobj, "numFeatures") + + + list( + coefficients = coefficients, + numFeatures = numFeatures + ) + }) + + +# Predicted values based on an LinearRegressionModel model + + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns the predicted values based on a LinearRegressionModel. +#' +#' @rdname spark.lm +#' @aliases predict,LinearRegressionModel,SparkDataFrame-method +#' @note predict(LinearRegressionModel) since 3.1.0 +setMethod("predict", signature(object = "LinearRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + + +# Save fitted LinearRegressionModel 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.lm +#' @aliases write.ml,LinearRegressionModel,character-method +#' @note write.ml(LinearRegressionModel, character) since 3.1.0 +setMethod("write.ml", signature(object = "LinearRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' Factorization Machines Regression Model +#' +#' \code{spark.fmRegressor} fits a factorization regression 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 \code{SparkDataFrame} of observations and labels for model fitting. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param factorSize dimensionality of the factors. +#' @param fitLinear whether to fit linear term. # TODO Can we express this with formula? +#' @param regParam the regularization parameter. +#' @param miniBatchFraction the mini-batch fraction parameter. +#' @param initStd the standard deviation of initial coefficients. +#' @param maxIter maximum iteration number. +#' @param stepSize stepSize parameter. +#' @param tol convergence tolerance of iterations. +#' @param solver solver parameter, supported options: "gd" (minibatch gradient descent) or "adamW". +#' @param seed seed parameter for weights initialization. +#' @param stringIndexerOrderType how to order categories of a string feature column. This is used to +#' decide the base level of a string feature as the last category +#' after ordering is dropped when encoding strings. Supported options +#' are "frequencyDesc", "frequencyAsc", "alphabetDesc", and +#' "alphabetAsc". The default value is "frequencyDesc". When the +#' ordering is set to "alphabetDesc", this drops the same category +#' as R when encoding strings. +#' @param ... additional arguments passed to the method. +#' @return \code{spark.fmRegressor} returns a fitted Factorization Machines Regression Model. +#' +#' @rdname spark.fmRegressor +#' @aliases spark.fmRegressor,SparkDataFrame,formula-method +#' @name spark.fmRegressor +#' @seealso \link{read.ml} +#' @examples +#' \dontrun{ +#' df <- read.df("data/mllib/sample_linear_regression_data.txt", source = "libsvm") +#' +#' # fit Factorization Machines Regression Model +#' model <- spark.fmRegressor( +#' df, label ~ features, +#' regParam = 0.01, maxIter = 10, fitLinear = TRUE +#' ) +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.fmRegressor since 3.1.0 +setMethod("spark.fmRegressor", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, factorSize = 8, fitLinear = TRUE, regParam = 0.0, + miniBatchFraction = 1.0, initStd = 0.01, maxIter = 100, stepSize=1.0, + tol = 1e-6, solver = c("adamW", "gd"), seed = NULL, + stringIndexerOrderType = c("frequencyDesc", "frequencyAsc", + "alphabetDesc", "alphabetAsc")) { + + + formula <- paste(deparse(formula), collapse = "") + + + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + + + solver <- match.arg(solver) + stringIndexerOrderType <- match.arg(stringIndexerOrderType) + + + jobj <- callJStatic("org.apache.spark.ml.r.FMRegressorWrapper", + "fit", + data@sdf, + formula, + as.integer(factorSize), + as.logical(fitLinear), + as.numeric(regParam), + as.numeric(miniBatchFraction), + as.numeric(initStd), + as.integer(maxIter), + as.numeric(stepSize), + as.numeric(tol), + solver, + seed, + stringIndexerOrderType) + new("FMRegressionModel", jobj = jobj) + }) + + +# Returns the summary of a FM Regression model produced by \code{spark.fmRegressor} + + +#' @param object a FM Regression Model model fitted by \code{spark.fmRegressor}. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' +#' @rdname spark.fmRegressor +#' @note summary(FMRegressionModel) since 3.1.0 +setMethod("summary", signature(object = "FMRegressionModel"), + function(object) { + jobj <- object@jobj + features <- callJMethod(jobj, "rFeatures") + coefficients <- callJMethod(jobj, "rCoefficients") + coefficients <- as.matrix(unlist(coefficients)) + colnames(coefficients) <- c("Estimate") + rownames(coefficients) <- unlist(features) + numFeatures <- callJMethod(jobj, "numFeatures") + raw_factors <- unlist(callJMethod(jobj, "rFactors")) + factor_size <- callJMethod(jobj, "factorSize") + + + list( + coefficients = coefficients, + factors = matrix(raw_factors, ncol = factor_size), + numFeatures = numFeatures, + factorSize = factor_size + ) + }) + + +# Predicted values based on an FMRegressionModel model + + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns the predicted values based on an FMRegressionModel. +#' +#' @rdname spark.fmRegressor +#' @aliases predict,FMRegressionModel,SparkDataFrame-method +#' @note predict(FMRegressionModel) since 3.1.0 +setMethod("predict", signature(object = "FMRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + + +# Save fitted FMRegressionModel 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.fmRegressor +#' @aliases write.ml,FMRegressionModel,character-method +#' @note write.ml(FMRegressionModel, character) since 3.1.0 +setMethod("write.ml", signature(object = "FMRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) diff --git a/R/pkg/R/mllib_utils.R b/R/pkg/R/mllib_utils.R index 7d04bffcba3a4..f38f1ac3a6b4c 100644 --- a/R/pkg/R/mllib_utils.R +++ b/R/pkg/R/mllib_utils.R @@ -123,6 +123,12 @@ read.ml <- function(path) { new("LinearSVCModel", jobj = jobj) } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.FPGrowthWrapper")) { new("FPGrowthModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.FMClassifierWrapper")) { + new("FMClassificationModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.LinearRegressionWrapper")) { + new("LinearRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.FMRegressorWrapper")) { + new("FMRegressionModel", jobj = jobj) } else { stop("Unsupported model: ", jobj) } diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index cdb59093781fb..cc8c92b8ab262 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -88,6 +88,49 @@ sparkR.stop <- function() { sparkR.session.stop() } +#' (Deprecated) Initialize a new Spark Context +#' +#' This function initializes a new SparkContext. +#' +#' @param master The Spark master URL +#' @param appName Application name to register with cluster manager +#' @param sparkHome Spark Home directory +#' @param sparkEnvir Named list of environment variables to set on worker nodes +#' @param sparkExecutorEnv Named list of environment variables to be used when launching executors +#' @param sparkJars Character vector of jar files to pass to the worker nodes +#' @param sparkPackages Character vector of package coordinates +#' @seealso \link{sparkR.session} +#' @rdname sparkR.init-deprecated +#' @examples +#'\dontrun{ +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark") +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark", +#' list(spark.executor.memory="1g")) +#' sc <- sparkR.init("yarn-client", "SparkR", "/home/spark", +#' list(spark.executor.memory="4g"), +#' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"), +#' c("one.jar", "two.jar", "three.jar"), +#' c("com.databricks:spark-avro_2.11:2.0.1")) +#'} +#' @note sparkR.init since 1.4.0 +sparkR.init <- function( + master = "", + appName = "SparkR", + sparkHome = Sys.getenv("SPARK_HOME"), + sparkEnvir = list(), + sparkExecutorEnv = list(), + sparkJars = "", + sparkPackages = "") { + .Deprecated("sparkR.session") + sparkR.sparkContext(master, + appName, + sparkHome, + convertNamedListToEnv(sparkEnvir), + convertNamedListToEnv(sparkExecutorEnv), + sparkJars, + sparkPackages) +} + # Internal function to handle creating the SparkContext. sparkR.sparkContext <- function( master = "", @@ -229,6 +272,61 @@ sparkR.sparkContext <- function( sc } +#' (Deprecated) Initialize a new SQLContext +#' +#' This function creates a SparkContext from an existing JavaSparkContext and +#' then uses it to initialize a new SQLContext +#' +#' Starting SparkR 2.0, a SparkSession is initialized and returned instead. +#' This API is deprecated and kept for backward compatibility only. +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @seealso \link{sparkR.session} +#' @rdname sparkRSQL.init-deprecated +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRSQL.init(sc) +#'} +#' @note sparkRSQL.init since 1.4.0 +sparkRSQL.init <- function(jsc = NULL) { + .Deprecated("sparkR.session") + + if (exists(".sparkRsession", envir = .sparkREnv)) { + return(get(".sparkRsession", envir = .sparkREnv)) + } + + # Default to without Hive support for backward compatibility. + sparkR.session(enableHiveSupport = FALSE) +} + +#' (Deprecated) Initialize a new HiveContext +#' +#' This function creates a HiveContext from an existing JavaSparkContext +#' +#' Starting SparkR 2.0, a SparkSession is initialized and returned instead. +#' This API is deprecated and kept for backward compatibility only. +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @seealso \link{sparkR.session} +#' @rdname sparkRHive.init-deprecated +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRHive.init(sc) +#'} +#' @note sparkRHive.init since 1.4.0 +sparkRHive.init <- function(jsc = NULL) { + .Deprecated("sparkR.session") + + if (exists(".sparkRsession", envir = .sparkREnv)) { + return(get(".sparkRsession", envir = .sparkREnv)) + } + + # Default to without Hive support for backward compatibility. + sparkR.session(enableHiveSupport = TRUE) +} + #' Get the existing SparkSession or initialize a new SparkSession. #' #' SparkSession is the entry point into SparkR. \code{sparkR.session} gets the existing diff --git a/R/pkg/tests/fulltests/test_mllib_classification.R b/R/pkg/tests/fulltests/test_mllib_classification.R index 2da3a022f8941..9dd275a173089 100644 --- a/R/pkg/tests/fulltests/test_mllib_classification.R +++ b/R/pkg/tests/fulltests/test_mllib_classification.R @@ -488,4 +488,38 @@ test_that("spark.naiveBayes", { expect_equal(class(collect(predictions)$clicked[1]), "character") }) +test_that("spark.fmClassifier", { + df <- withColumn( + suppressWarnings(createDataFrame(iris)), + "Species", otherwise(when(column("Species") == "Setosa", "Setosa"), "Not-Setosa") + ) + + model1 <- spark.fmClassifier( + df, Species ~ ., + regParam = 0.01, maxIter = 10, fitLinear = TRUE, factorSize = 3 + ) + + prediction1 <- predict(model1, df) + expect_is(prediction1, "SparkDataFrame") + expect_equal(summary(model1)$factorSize, 3) + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-fmclassifier", fileext = ".tmp") + write.ml(model1, modelPath) + model2 <- read.ml(modelPath) + + expect_is(model2, "FMClassificationModel") + + expect_equal(summary(model1), summary(model2)) + + prediction2 <- predict(model2, df) + expect_equal( + collect(drop(prediction1, c("rawPrediction", "probability"))), + collect(drop(prediction2, c("rawPrediction", "probability"))) + ) + unlink(modelPath) + } +}) + sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_regression.R b/R/pkg/tests/fulltests/test_mllib_regression.R index b40c4cb9a9694..0f2a62b7229ca 100644 --- a/R/pkg/tests/fulltests/test_mllib_regression.R +++ b/R/pkg/tests/fulltests/test_mllib_regression.R @@ -549,6 +549,65 @@ test_that("spark.survreg", { coefs <- as.vector(summary(model)$coefficients[, 1]) expect_true(all(abs(rCoefs[o] - coefs[o]) < 1e-4)) } + + test_that("spark.lm", { + df <- suppressWarnings(createDataFrame(iris)) + + model <- spark.lm( + df, Sepal_Width ~ ., + regParam = 0.01, maxIter = 10 + ) + + prediction1 <- predict(model, df) + expect_is(prediction1, "SparkDataFrame") + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-lm", fileext = ".tmp") + write.ml(model, modelPath) + model2 <- read.ml(modelPath) + + expect_is(model2, "LinearRegressionModel") + expect_equal(summary(model), summary(model2)) + + prediction2 <- predict(model2, df) + expect_equal( + collect(prediction1), + collect(prediction2) + ) + unlink(modelPath) + } + }) +}) + + +test_that("spark.fmRegressor", { + df <- suppressWarnings(createDataFrame(iris)) + + model <- spark.fmRegressor( + df, Sepal_Width ~ ., + regParam = 0.01, maxIter = 10, fitLinear = TRUE + ) + + prediction1 <- predict(model, df) + expect_is(prediction1, "SparkDataFrame") + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-fmregressor", fileext = ".tmp") + write.ml(model, modelPath) + model2 <- read.ml(modelPath) + + expect_is(model2, "FMRegressionModel") + expect_equal(summary(model), summary(model2)) + + prediction2 <- predict(model2, df) + expect_equal( + collect(prediction1), + collect(prediction2) + ) + unlink(modelPath) + } }) sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 0f26184fa3e9e..c892feb61da8b 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -106,6 +106,15 @@ if (is_windows()) { Sys.setenv(TZ = "GMT") } +test_that("calling sparkRSQL.init returns existing SQL context", { + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) + expect_equal(suppressWarnings(sparkRSQL.init(sc)), sqlContext) +}) + +test_that("calling sparkRSQL.init returns existing SparkSession", { + expect_equal(suppressWarnings(sparkRSQL.init(sc)), sparkSession) +}) + test_that("calling sparkR.session returns existing SparkSession", { expect_equal(sparkR.session(), sparkSession) }) @@ -656,10 +665,10 @@ test_that("test tableNames and tables", { expect_true("tableName" %in% colnames(tables())) expect_true(all(c("tableName", "database", "isTemporary") %in% colnames(tables()))) - createOrReplaceTempView(df, "table2") + suppressWarnings(registerTempTable(df, "table2")) tables <- listTables() expect_equal(count(tables), count + 2) - dropTempView("table1") + suppressWarnings(dropTempTable("table1")) expect_true(dropTempView("table2")) tables <- listTables() @@ -1772,6 +1781,28 @@ test_that("column functions", { collect(select(df, alias(not(df$is_true), "is_false"))), data.frame(is_false = c(FALSE, TRUE, NA)) ) + + # Test percentile_approx + actual <- lapply( + list( + percentile_approx(column("foo"), 0.5), + percentile_approx(column("bar"), lit(0.25), lit(42L)), + percentile_approx(column("bar"), c(0.25, 0.5, 0.75)), + percentile_approx(column("foo"), c(0.05, 0.95), 100L), + percentile_approx("foo", c(0.5)), + percentile_approx("bar", c(0.1, 0.9), 10L)), + function(x) SparkR:::callJMethod(x@jc, "toString")) + + expected <- list( + "percentile_approx(foo, 0.5, 10000)", + "percentile_approx(bar, 0.25, 42)", + "percentile_approx(bar, array(0.25, 0.5, 0.75), 10000)", + "percentile_approx(foo, array(0.05, 0.95), 100)", + "percentile_approx(foo, 0.5, 10000)", + "percentile_approx(bar, array(0.1, 0.9), 10)" + ) + + expect_equal(actual, expected) }) test_that("column binary mathfunctions", { diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index 9e48ae34634e2..3713e6c784855 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -523,6 +523,8 @@ SparkR supports the following machine learning models and algorithms. * Naive Bayes +* Factorization Machines (FM) Classifier + #### Regression * Accelerated Failure Time (AFT) Survival Model @@ -531,6 +533,10 @@ SparkR supports the following machine learning models and algorithms. * Isotonic Regression +* Linear Regression + +* Factorization Machines (FM) Regressor + #### Tree - Classification and Regression * Decision Tree @@ -705,6 +711,24 @@ naiveBayesPrediction <- predict(naiveBayesModel, titanicDF) head(select(naiveBayesPrediction, "Class", "Sex", "Age", "Survived", "prediction")) ``` +#### Factorization Machines Classifier + +Factorization Machines for classification problems. + +For background and details about the implementation of factorization machines, +refer to the [Factorization Machines section](https://spark.apache.org/docs/latest/ml-classification-regression.html#factorization-machines). + +```{r} +t <- as.data.frame(Titanic) +training <- createDataFrame(t) + +model <- spark.fmClassifier(training, Survived ~ Age + Sex) +summary(model) + +predictions <- predict(model, training) +head(select(predictions, predictions$prediction)) +``` + #### Accelerated Failure Time Survival Model Survival analysis studies the expected duration of time until an event happens, and often the relationship with risk factors or treatment taken on the subject. In contrast to standard regression analysis, survival modeling has to deal with special characteristics in the data including non-negative survival time and censoring. @@ -813,6 +837,32 @@ newDF <- createDataFrame(data.frame(x = c(1.5, 3.2))) head(predict(isoregModel, newDF)) ``` +#### Linear Regression + +Linear regression model. + +```{r} +model <- spark.lm(carsDF, mpg ~ wt + hp) + +summary(model) +predictions <- predict(model, carsDF) +head(select(predictions, predictions$prediction)) +``` + +#### Factorization Machines Regressor + +Factorization Machines for regression problems. + +For background and details about the implementation of factorization machines, +refer to the [Factorization Machines section](https://spark.apache.org/docs/latest/ml-classification-regression.html#factorization-machines). + +```{r} +model <- spark.fmRegressor(carsDF, mpg ~ wt + hp) +summary(model) +predictions <- predict(model, carsDF) +head(select(predictions, predictions$prediction)) +``` + #### Decision Tree `spark.decisionTree` fits a [decision tree](https://en.wikipedia.org/wiki/Decision_tree_learning) classification or regression model on a `SparkDataFrame`. diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index 68fafbb848001..57b86254ab424 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -248,6 +248,10 @@ Examples: - Build and push image with tag "v2.3.0" to docker.io/myrepo $0 -r docker.io/myrepo -t v2.3.0 build $0 -r docker.io/myrepo -t v2.3.0 push + + - Build and push JDK11-based image with tag "v3.0.0" to docker.io/myrepo + $0 -r docker.io/myrepo -t v3.0.0 -b java_image_tag=11-jre-slim build + $0 -r docker.io/myrepo -t v3.0.0 push EOF } diff --git a/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java index d99b9bdbce392..a0de9df1986f5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java @@ -123,7 +123,7 @@ public TransportContext( if (conf.getModuleName() != null && conf.getModuleName().equalsIgnoreCase("shuffle") && - !isClientOnly) { + !isClientOnly && conf.separateChunkFetchRequest()) { chunkFetchWorkers = NettyUtils.createEventLoop( IOMode.valueOf(conf.ioMode()), conf.chunkFetchHandlerThreads(), @@ -187,8 +187,6 @@ public TransportChannelHandler initializePipeline( RpcHandler channelRpcHandler) { try { TransportChannelHandler channelHandler = createChannelHandler(channel, channelRpcHandler); - ChunkFetchRequestHandler chunkFetchHandler = - createChunkFetchHandler(channelHandler, channelRpcHandler); ChannelPipeline pipeline = channel.pipeline() .addLast("encoder", ENCODER) .addLast(TransportFrameDecoder.HANDLER_NAME, NettyUtils.createFrameDecoder()) @@ -200,6 +198,9 @@ public TransportChannelHandler initializePipeline( .addLast("handler", channelHandler); // Use a separate EventLoopGroup to handle ChunkFetchRequest messages for shuffle rpcs. if (chunkFetchWorkers != null) { + ChunkFetchRequestHandler chunkFetchHandler = new ChunkFetchRequestHandler( + channelHandler.getClient(), rpcHandler.getStreamManager(), + conf.maxChunksBeingTransferred(), true /* syncModeEnabled */); pipeline.addLast(chunkFetchWorkers, "chunkFetchHandler", chunkFetchHandler); } return channelHandler; @@ -217,19 +218,17 @@ public TransportChannelHandler initializePipeline( private TransportChannelHandler createChannelHandler(Channel channel, RpcHandler rpcHandler) { TransportResponseHandler responseHandler = new TransportResponseHandler(channel); TransportClient client = new TransportClient(channel, responseHandler); + boolean separateChunkFetchRequest = conf.separateChunkFetchRequest(); + ChunkFetchRequestHandler chunkFetchRequestHandler = null; + if (!separateChunkFetchRequest) { + chunkFetchRequestHandler = new ChunkFetchRequestHandler( + client, rpcHandler.getStreamManager(), + conf.maxChunksBeingTransferred(), false /* syncModeEnabled */); + } TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, - rpcHandler, conf.maxChunksBeingTransferred()); + rpcHandler, conf.maxChunksBeingTransferred(), chunkFetchRequestHandler); return new TransportChannelHandler(client, responseHandler, requestHandler, - conf.connectionTimeoutMs(), closeIdleConnections, this); - } - - /** - * Creates the dedicated ChannelHandler for ChunkFetchRequest messages. - */ - private ChunkFetchRequestHandler createChunkFetchHandler(TransportChannelHandler channelHandler, - RpcHandler rpcHandler) { - return new ChunkFetchRequestHandler(channelHandler.getClient(), - rpcHandler.getStreamManager(), conf.maxChunksBeingTransferred()); + conf.connectionTimeoutMs(), separateChunkFetchRequest, closeIdleConnections, this); } public TransportConf getConf() { return conf; } diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index c9ef9f918ffd1..24c436a504fa8 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicReference; import com.codahale.metrics.MetricSet; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -61,6 +62,7 @@ public class TransportClientFactory implements Closeable { private static class ClientPool { TransportClient[] clients; Object[] locks; + volatile long lastConnectionFailed; ClientPool(int size) { clients = new TransportClient[size]; @@ -68,6 +70,7 @@ private static class ClientPool { for (int i = 0; i < size; i++) { locks[i] = new Object(); } + lastConnectionFailed = 0; } } @@ -86,6 +89,7 @@ private static class ClientPool { private EventLoopGroup workerGroup; private final PooledByteBufAllocator pooledAllocator; private final NettyMemoryMetrics metrics; + private final int fastFailTimeWindow; public TransportClientFactory( TransportContext context, @@ -112,6 +116,7 @@ public TransportClientFactory( } this.metrics = new NettyMemoryMetrics( this.pooledAllocator, conf.getModuleName() + "-client", conf); + fastFailTimeWindow = (int)(conf.ioRetryWaitTimeMs() * 0.95); } public MetricSet getAllMetrics() { @@ -121,18 +126,27 @@ public MetricSet getAllMetrics() { /** * Create a {@link TransportClient} connecting to the given remote host / port. * - * We maintains an array of clients (size determined by spark.shuffle.io.numConnectionsPerPeer) + * We maintain an array of clients (size determined by spark.shuffle.io.numConnectionsPerPeer) * and randomly picks one to use. If no client was previously created in the randomly selected * spot, this function creates a new client and places it there. * + * If the fastFail parameter is true, fail immediately when the last attempt to the same address + * failed within the fast fail time window (95 percent of the io wait retry timeout). The + * assumption is the caller will handle retrying. + * * Prior to the creation of a new TransportClient, we will execute all * {@link TransportClientBootstrap}s that are registered with this factory. * * This blocks until a connection is successfully established and fully bootstrapped. * * Concurrency: This method is safe to call from multiple threads. + * + * @param remoteHost remote address host + * @param remotePort remote address port + * @param fastFail whether this call should fail immediately when the last attempt to the same + * address failed with in the last fast fail time window. */ - public TransportClient createClient(String remoteHost, int remotePort) + public TransportClient createClient(String remoteHost, int remotePort, boolean fastFail) throws IOException, InterruptedException { // Get connection from the connection pool first. // If it is not found or not active, create a new one. @@ -192,11 +206,30 @@ public TransportClient createClient(String remoteHost, int remotePort) logger.info("Found inactive connection to {}, creating a new one.", resolvedAddress); } } - clientPool.clients[clientIndex] = createClient(resolvedAddress); + // If this connection should fast fail when last connection failed in last fast fail time + // window and it did, fail this connection directly. + if (fastFail && System.currentTimeMillis() - clientPool.lastConnectionFailed < + fastFailTimeWindow) { + throw new IOException( + String.format("Connecting to %s failed in the last %s ms, fail this connection directly", + resolvedAddress, fastFailTimeWindow)); + } + try { + clientPool.clients[clientIndex] = createClient(resolvedAddress); + clientPool.lastConnectionFailed = 0; + } catch (IOException e) { + clientPool.lastConnectionFailed = System.currentTimeMillis(); + throw e; + } return clientPool.clients[clientIndex]; } } + public TransportClient createClient(String remoteHost, int remotePort) + throws IOException, InterruptedException { + return createClient(remoteHost, remotePort, false); + } + /** * Create a completely new {@link TransportClient} to the given remote host / port. * This connection is not pooled. @@ -210,7 +243,8 @@ public TransportClient createUnmanagedClient(String remoteHost, int remotePort) } /** Create a completely new {@link TransportClient} to the remote address. */ - private TransportClient createClient(InetSocketAddress address) + @VisibleForTesting + TransportClient createClient(InetSocketAddress address) throws IOException, InterruptedException { logger.debug("Creating new connection to {}", address); diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java index 94412c4db559f..82810dacdad84 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java @@ -55,14 +55,17 @@ public class ChunkFetchRequestHandler extends SimpleChannelInboundHandler { + ChannelFuture channelFuture; + if (syncModeEnabled) { + channelFuture = channel.writeAndFlush(result).await(); + } else { + channelFuture = channel.writeAndFlush(result); + } + return channelFuture.addListener((ChannelFutureListener) future -> { if (future.isSuccess()) { logger.trace("Sent result {} to client {}", result, remoteAddress); } else { diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java index 31371f6970ffe..e53a0c1a0852e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -58,6 +58,7 @@ public class TransportChannelHandler extends SimpleChannelInboundHandler { /** The max number of chunks being transferred and not finished yet. */ private final long maxChunksBeingTransferred; + /** The dedicated ChannelHandler for ChunkFetchRequest messages. */ + private final ChunkFetchRequestHandler chunkFetchRequestHandler; + public TransportRequestHandler( Channel channel, TransportClient reverseClient, RpcHandler rpcHandler, - Long maxChunksBeingTransferred) { + Long maxChunksBeingTransferred, + ChunkFetchRequestHandler chunkFetchRequestHandler) { this.channel = channel; this.reverseClient = reverseClient; this.rpcHandler = rpcHandler; this.streamManager = rpcHandler.getStreamManager(); this.maxChunksBeingTransferred = maxChunksBeingTransferred; + this.chunkFetchRequestHandler = chunkFetchRequestHandler; } @Override @@ -97,8 +102,10 @@ public void channelInactive() { } @Override - public void handle(RequestMessage request) { - if (request instanceof RpcRequest) { + public void handle(RequestMessage request) throws Exception { + if (request instanceof ChunkFetchRequest) { + chunkFetchRequestHandler.processFetchRequest(channel, (ChunkFetchRequest) request); + } else if (request instanceof RpcRequest) { processRpcRequest((RpcRequest) request); } else if (request instanceof OneWayMessage) { processOneWayMessage((OneWayMessage) request); 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 cc0f2919568ac..6c37f9a382376 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 @@ -316,7 +316,8 @@ public long maxChunksBeingTransferred() { /** * Percentage of io.serverThreads used by netty to process ChunkFetchRequest. - * Shuffle server will use a separate EventLoopGroup to process ChunkFetchRequest messages. + * When the config `spark.shuffle.server.chunkFetchHandlerThreadsPercent` is set, + * shuffle server will use a separate EventLoopGroup to process ChunkFetchRequest messages. * Although when calling the async writeAndFlush on the underlying channel to send * response back to client, the I/O on the channel is still being handled by * {@link org.apache.spark.network.server.TransportServer}'s default EventLoopGroup @@ -339,12 +340,20 @@ public int chunkFetchHandlerThreads() { return 0; } int chunkFetchHandlerThreadsPercent = - conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 100); + Integer.parseInt(conf.get("spark.shuffle.server.chunkFetchHandlerThreadsPercent")); int threads = this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors(); return (int) Math.ceil(threads * (chunkFetchHandlerThreadsPercent / 100.0)); } + /** + * Whether to use a separate EventLoopGroup to process ChunkFetchRequest messages, it is decided + * by the config `spark.shuffle.server.chunkFetchHandlerThreadsPercent` is set or not. + */ + public boolean separateChunkFetchRequest() { + return conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 0) > 0; + } + /** * Whether to use the old protocol while doing the shuffle block fetching. * It is only enabled while we need the compatibility in the scenario of new spark version diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java index 7e30ed4048ca8..addb4ff332746 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java @@ -22,7 +22,6 @@ import java.util.List; import io.netty.channel.Channel; -import org.apache.spark.network.server.ChunkFetchRequestHandler; import org.junit.Assert; import org.junit.Test; @@ -33,6 +32,7 @@ import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.protocol.*; +import org.apache.spark.network.server.ChunkFetchRequestHandler; import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; @@ -68,7 +68,7 @@ public void handleChunkFetchRequest() throws Exception { long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); TransportClient reverseClient = mock(TransportClient.class); ChunkFetchRequestHandler requestHandler = new ChunkFetchRequestHandler(reverseClient, - rpcHandler.getStreamManager(), 2L); + rpcHandler.getStreamManager(), 2L, false); RequestMessage request0 = new ChunkFetchRequest(new StreamChunkId(streamId, 0)); requestHandler.channelRead(context, request0); diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java index a43a659048686..0a6447176237a 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java @@ -39,7 +39,7 @@ public class TransportRequestHandlerSuite { @Test - public void handleStreamRequest() { + public void handleStreamRequest() throws Exception { RpcHandler rpcHandler = new NoOpRpcHandler(); OneForOneStreamManager streamManager = (OneForOneStreamManager) (rpcHandler.getStreamManager()); Channel channel = mock(Channel.class); @@ -66,7 +66,7 @@ public void handleStreamRequest() { TransportClient reverseClient = mock(TransportClient.class); TransportRequestHandler requestHandler = new TransportRequestHandler(channel, reverseClient, - rpcHandler, 2L); + rpcHandler, 2L, null); RequestMessage request0 = new StreamRequest(String.format("%d_%d", streamId, 0)); requestHandler.handle(request0); diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/common/network-common/src/test/java/org/apache/spark/network/client/TransportClientFactorySuite.java similarity index 88% rename from common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java rename to common/network-common/src/test/java/org/apache/spark/network/client/TransportClientFactorySuite.java index 9b76981c31c57..ea0ac51589dc3 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/client/TransportClientFactorySuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.network; +package org.apache.spark.network.client; import java.io.IOException; import java.util.Collections; @@ -29,14 +29,16 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.TransportContext; import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; @@ -224,4 +226,24 @@ public void closeFactoryBeforeCreateClient() throws IOException, InterruptedExce factory.close(); factory.createClient(TestUtils.getLocalHost(), server1.getPort()); } + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void fastFailConnectionInTimeWindow() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + TransportServer server = context.createServer(); + int unreachablePort = server.getPort(); + server.close(); + try { + factory.createClient(TestUtils.getLocalHost(), unreachablePort, true); + } catch (Exception e) { + assert(e instanceof IOException); + } + expectedException.expect(IOException.class); + expectedException.expectMessage("fail this connection directly"); + factory.createClient(TestUtils.getLocalHost(), unreachablePort, true); + expectedException = ExpectedException.none(); + } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index d6185f089d3c0..51dc3337261b9 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -101,11 +101,12 @@ public void fetchBlocks( checkInit(); logger.debug("External shuffle fetch from {}:{} (executor id {})", host, port, execId); try { + int maxRetries = conf.maxIORetries(); RetryingBlockFetcher.BlockFetchStarter blockFetchStarter = (blockIds1, listener1) -> { // Unless this client is closed. if (clientFactory != null) { - TransportClient client = clientFactory.createClient(host, port); + TransportClient client = clientFactory.createClient(host, port, maxRetries > 0); new OneForOneBlockFetcher(client, appId, execId, blockIds1, listener1, conf, downloadFileManager).start(); } else { @@ -113,7 +114,6 @@ public void fetchBlocks( } }; - int maxRetries = conf.maxIORetries(); if (maxRetries > 0) { // Note this Fetcher will correctly handle maxRetries == 0; we avoid it just in case there's // a bug in this code. We should remove the if statement once we're sure of the stability. 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 815a56d765b6a..c41efbad8ffec 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 @@ -188,7 +188,7 @@ protected void serviceInit(Configuration conf) throws Exception { int port = conf.getInt( SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); - transportContext = new TransportContext(transportConf, blockHandler); + transportContext = new TransportContext(transportConf, blockHandler, true); shuffleServer = transportContext.createServer(port, bootstraps); // the port should normally be fixed, but for tests its useful to find an open port port = shuffleServer.getPort(); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index c5384669eb922..186597fa64780 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1105,6 +1105,10 @@ public static class IntWrapper implements Serializable { * @return true if the parsing was successful else false */ public boolean toLong(LongWrapper toLongResult) { + return toLong(toLongResult, true); + } + + private boolean toLong(LongWrapper toLongResult, boolean allowDecimal) { int offset = 0; while (offset < this.numBytes && getByte(offset) <= ' ') offset++; if (offset == this.numBytes) return false; @@ -1129,7 +1133,7 @@ public boolean toLong(LongWrapper toLongResult) { while (offset <= end) { b = getByte(offset); offset++; - if (b == separator) { + if (b == separator && allowDecimal) { // We allow decimals and will return a truncated integral in that case. // Therefore we won't throw an exception here (checking the fractional // part happens below.) @@ -1198,6 +1202,10 @@ public boolean toLong(LongWrapper toLongResult) { * @return true if the parsing was successful else false */ public boolean toInt(IntWrapper intWrapper) { + return toInt(intWrapper, true); + } + + private boolean toInt(IntWrapper intWrapper, boolean allowDecimal) { int offset = 0; while (offset < this.numBytes && getByte(offset) <= ' ') offset++; if (offset == this.numBytes) return false; @@ -1222,7 +1230,7 @@ public boolean toInt(IntWrapper intWrapper) { while (offset <= end) { b = getByte(offset); offset++; - if (b == separator) { + if (b == separator && allowDecimal) { // We allow decimals and will return a truncated integral in that case. // Therefore we won't throw an exception here (checking the fractional // part happens below.) @@ -1276,9 +1284,7 @@ public boolean toShort(IntWrapper intWrapper) { if (toInt(intWrapper)) { int intValue = intWrapper.value; short result = (short) intValue; - if (result == intValue) { - return true; - } + return result == intValue; } return false; } @@ -1287,9 +1293,7 @@ public boolean toByte(IntWrapper intWrapper) { if (toInt(intWrapper)) { int intValue = intWrapper.value; byte result = (byte) intValue; - if (result == intValue) { - return true; - } + return result == intValue; } return false; } @@ -1302,7 +1306,7 @@ public boolean toByte(IntWrapper intWrapper) { */ public long toLongExact() { LongWrapper result = new LongWrapper(); - if (toLong(result)) { + if (toLong(result, false)) { return result.value; } throw new NumberFormatException("invalid input syntax for type numeric: " + this); @@ -1316,7 +1320,7 @@ public long toLongExact() { */ public int toIntExact() { IntWrapper result = new IntWrapper(); - if (toInt(result)) { + if (toInt(result, false)) { return result.value; } throw new NumberFormatException("invalid input syntax for type numeric: " + this); diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 7bdd89488d119..a57cd3b3f3cbf 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -96,8 +96,7 @@ public final class BytesToBytesMap extends MemoryConsumer { * since that's the largest power-of-2 that's less than Integer.MAX_VALUE. We need two long array * entries per key, giving us a maximum capacity of (1 << 29). */ - @VisibleForTesting - static final int MAX_CAPACITY = (1 << 29); + public static final int MAX_CAPACITY = (1 << 29); // This choice of page table size and page size means that we can address up to 500 gigabytes // of memory. diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 2f70b47c80f26..93b37c296271b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -363,15 +363,24 @@ $(document).ready(function () { dataToShow.showBytesSpilledData = (responseBody.diskBytesSpilled > 0 || responseBody.memoryBytesSpilled > 0); + var columnIndicesToRemove = []; if (!dataToShow.showShuffleReadData) { $('#shuffle_read_blocked_time').remove(); $('#shuffle_remote_reads').remove(); - optionalColumns.splice(2, 2); + columnIndicesToRemove.push(2); + columnIndicesToRemove.push(3); } if (!dataToShow.showShuffleWriteData) { $('#shuffle_write_time').remove(); - optionalColumns.splice(7, 1) + columnIndicesToRemove.push(7); + } + + if (columnIndicesToRemove.length > 0) { + columnIndicesToRemove.sort(function(a, b) { return b - a; }); + columnIndicesToRemove.forEach(function(idx) { + optionalColumns.splice(idx, 1); + }); } // prepare data for executor summary table diff --git a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js index 6a944afe83a8e..6d4c8d94b4288 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js +++ b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js @@ -33,6 +33,8 @@ var yValueFormat = d3.format(",.2f"); var unitLabelYOffset = -10; +var onClickTimeline = function() {}; + // Show a tooltip "text" for "node" function showBootstrapTooltip(node, text) { $(node).tooltip({title: text, trigger: "manual", container: "body"}); @@ -44,6 +46,45 @@ function hideBootstrapTooltip(node) { $(node).tooltip("dispose"); } +// Return the function to scroll to the corresponding +// row on clicking a point of batch in the timeline. +function getOnClickTimelineFunction() { + // If the user click one point in the graphs, jump to the batch row and highlight it. And + // recovery the batch row after 3 seconds if necessary. + // We need to remember the last clicked batch so that we can recovery it. + var lastClickedBatch = null; + var lastTimeout = null; + + return function(d) { + var batchSelector = $("#batch-" + d.x); + // If there is a corresponding batch row, scroll down to it and highlight it. + if (batchSelector.length > 0) { + if (lastTimeout != null) { + window.clearTimeout(lastTimeout); + } + if (lastClickedBatch != null) { + clearBatchRow(lastClickedBatch); + lastClickedBatch = null; + } + lastClickedBatch = d.x; + highlightBatchRow(lastClickedBatch); + lastTimeout = window.setTimeout(function () { + lastTimeout = null; + if (lastClickedBatch != null) { + clearBatchRow(lastClickedBatch); + lastClickedBatch = null; + } + }, 3000); // Clean up after 3 seconds + + var topOffset = batchSelector.offset().top - 15; + if (topOffset < 0) { + topOffset = 0; + } + $('html,body').animate({scrollTop: topOffset}, 200); + } + } +} + // Register a timeline graph. All timeline graphs should be register before calling any // "drawTimeline" so that we can determine the max margin left for all timeline graphs. function registerTimeline(minY, maxY) { @@ -189,34 +230,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { .attr("opacity", function(d) { return isFailedBatch(d.x) ? "1" : "0";}) .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "3";}); }) - .on("click", function(d) { - var batchSelector = $("#batch-" + d.x); - // If there is a corresponding batch row, scroll down to it and highlight it. - if (batchSelector.length > 0) { - if (lastTimeout != null) { - window.clearTimeout(lastTimeout); - } - if (lastClickedBatch != null) { - clearBatchRow(lastClickedBatch); - lastClickedBatch = null; - } - lastClickedBatch = d.x; - highlightBatchRow(lastClickedBatch); - lastTimeout = window.setTimeout(function () { - lastTimeout = null; - if (lastClickedBatch != null) { - clearBatchRow(lastClickedBatch); - lastClickedBatch = null; - } - }, 3000); // Clean up after 3 seconds - - var topOffset = batchSelector.offset().top - 15; - if (topOffset < 0) { - topOffset = 0; - } - $('html,body').animate({scrollTop: topOffset}, 200); - } - }); + .on("click", onClickTimeline); } /** diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css index 3f31403eaeef3..c9bf83ca98b4f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css @@ -238,18 +238,6 @@ tr.corresponding-item-hover > td, tr.corresponding-item-hover > th { background-color: #D6FFE4 !important; } -#application-timeline.collapsed { - display: none; -} - -#job-timeline.collapsed { - display: none; -} - -#task-assignment-timeline.collapsed { - display: none; -} - .control-panel { margin-bottom: 5px; } diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index b2cd616791734..a63ee8696f906 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -26,7 +26,7 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { editable: false, align: 'left', showCurrentTime: false, - min: startTime, + start: startTime, zoomable: false, moment: function (date) { return vis.moment(date).utcOffset(offset); @@ -50,7 +50,7 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { }; $(this).click(function() { - var jobPagePath = $(getSelectorForJobEntry(this)).find("a.name-link").attr("href") + var jobPagePath = $(getSelectorForJobEntry(this)).find("a.name-link").attr("href"); window.location.href = jobPagePath }); @@ -75,6 +75,9 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { $("#application-timeline").toggleClass('collapsed'); + var visibilityState = status ? "" : "none"; + $("#application-timeline").css("display", visibilityState); + // Switch the class of the arrow from open to closed. $(this).find('.expand-application-timeline-arrow').toggleClass('arrow-open'); $(this).find('.expand-application-timeline-arrow').toggleClass('arrow-closed'); @@ -89,6 +92,8 @@ $(function () { // Set it to false so that the click function can revert it window.localStorage.setItem("expand-application-timeline", "false"); $("span.expand-application-timeline").trigger('click'); + } else { + $("#application-timeline").css("display", "none"); } }); @@ -103,7 +108,7 @@ function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { editable: false, align: 'left', showCurrentTime: false, - min: startTime, + start: startTime, zoomable: false, moment: function (date) { return vis.moment(date).utcOffset(offset); @@ -152,6 +157,9 @@ function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { $("#job-timeline").toggleClass('collapsed'); + var visibilityState = status ? "" : "none"; + $("#job-timeline").css("display", visibilityState); + // Switch the class of the arrow from open to closed. $(this).find('.expand-job-timeline-arrow').toggleClass('arrow-open'); $(this).find('.expand-job-timeline-arrow').toggleClass('arrow-closed'); @@ -166,13 +174,15 @@ $(function () { // Set it to false so that the click function can revert it window.localStorage.setItem("expand-job-timeline", "false"); $("span.expand-job-timeline").trigger('click'); + } else { + $("#job-timeline").css("display", "none"); } }); function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, maxFinishTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); - var container = $("#task-assignment-timeline")[0] + var container = $("#task-assignment-timeline")[0]; var options = { groupOrder: function(a, b) { return a.value - b.value @@ -181,15 +191,15 @@ function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, ma align: 'left', selectable: false, showCurrentTime: false, - min: minLaunchTime, - max: maxFinishTime, + start: minLaunchTime, + end: maxFinishTime, zoomable: false, moment: function (date) { return vis.moment(date).utcOffset(offset); } }; - var taskTimeline = new vis.Timeline(container) + var taskTimeline = new vis.Timeline(container); taskTimeline.setOptions(options); taskTimeline.setGroups(groups); taskTimeline.setItems(items); @@ -220,6 +230,9 @@ function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, ma $("#task-assignment-timeline").toggleClass("collapsed"); + var visibilityState = status ? "" : "none"; + $("#task-assignment-timeline").css("display", visibilityState); + // Switch the class of the arrow from open to closed. $(this).find(".expand-task-assignment-timeline-arrow").toggleClass("arrow-open"); $(this).find(".expand-task-assignment-timeline-arrow").toggleClass("arrow-closed"); @@ -234,6 +247,8 @@ $(function () { // Set it to false so that the click function can revert it window.localStorage.setItem("expand-task-assignment-timeline", "false"); $("span.expand-task-assignment-timeline").trigger('click'); + } else { + $("#task-assignment-timeline").css("display", "none"); } }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css new file mode 100644 index 0000000000000..d22a9ac801891 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css @@ -0,0 +1,2 @@ +.vis [class*=span]{min-height:0;width:auto}.vis .overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-active{box-shadow:0 0 10px #86d5f8}div.vis-configuration{position:relative;display:block;float:left;font-size:12px}div.vis-configuration-wrapper{display:block;width:700px}div.vis-configuration-wrapper:after{clear:both;content:"";display:block}div.vis-configuration.vis-config-option-container{display:block;width:495px;background-color:#fff;border:2px solid #f7f8fa;border-radius:4px;margin-top:20px;left:10px;padding-left:5px}div.vis-configuration.vis-config-button{display:block;width:495px;height:25px;vertical-align:middle;line-height:25px;background-color:#f7f8fa;border:2px solid #ceced0;border-radius:4px;margin-top:20px;left:10px;padding-left:5px;cursor:pointer;margin-bottom:30px}div.vis-configuration.vis-config-button.hover{background-color:#4588e6;border:2px solid #214373;color:#fff}div.vis-configuration.vis-config-item{display:block;float:left;width:495px;height:25px;vertical-align:middle;line-height:25px}div.vis-configuration.vis-config-item.vis-config-s2{left:10px;background-color:#f7f8fa;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-item.vis-config-s3{left:20px;background-color:#e4e9f0;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-item.vis-config-s4{left:30px;background-color:#cfd8e6;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-header{font-size:18px;font-weight:700}div.vis-configuration.vis-config-label{width:120px;height:25px;line-height:25px}div.vis-configuration.vis-config-label.vis-config-s3{width:110px}div.vis-configuration.vis-config-label.vis-config-s4{width:100px}div.vis-configuration.vis-config-colorBlock{top:1px;width:30px;height:19px;border:1px solid #444;border-radius:2px;padding:0;margin:0;cursor:pointer}input.vis-configuration.vis-config-checkbox{left:-5px}input.vis-configuration.vis-config-rangeinput{position:relative;top:-5px;width:60px;padding:1px;margin:0;pointer-events:none}input.vis-configuration.vis-config-range{-webkit-appearance:none;border:0 solid #fff;background-color:transparent;width:300px;height:20px}input.vis-configuration.vis-config-range::-webkit-slider-runnable-track{width:300px;height:5px;background:#dedede;background:-moz-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#dedede),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#dedede,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(180deg,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr="#dedede",endColorstr="#c8c8c8",GradientType=0);border:1px solid #999;box-shadow:0 0 3px 0 #aaa;border-radius:3px}input.vis-configuration.vis-config-range::-webkit-slider-thumb{-webkit-appearance:none;border:1px solid #14334b;height:17px;width:17px;border-radius:50%;background:#3876c2;background:-moz-linear-gradient(top,#3876c2 0,#385380 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#3876c2),color-stop(100%,#385380));background:-webkit-linear-gradient(top,#3876c2,#385380);background:-o-linear-gradient(top,#3876c2 0,#385380 100%);background:-ms-linear-gradient(top,#3876c2 0,#385380 100%);background:linear-gradient(180deg,#3876c2 0,#385380);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr="#3876c2",endColorstr="#385380",GradientType=0);box-shadow:0 0 1px 0 #111927;margin-top:-7px}input.vis-configuration.vis-config-range:focus{outline:none}input.vis-configuration.vis-config-range:focus::-webkit-slider-runnable-track{background:#9d9d9d;background:-moz-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#9d9d9d),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#9d9d9d,#c8c8c8 99%);background:-o-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:linear-gradient(180deg,#9d9d9d 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr="#9d9d9d",endColorstr="#c8c8c8",GradientType=0)}input.vis-configuration.vis-config-range::-moz-range-track{width:300px;height:10px;background:#dedede;background:-moz-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#dedede),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#dedede,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(180deg,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr="#dedede",endColorstr="#c8c8c8",GradientType=0);border:1px solid #999;box-shadow:0 0 3px 0 #aaa;border-radius:3px}input.vis-configuration.vis-config-range::-moz-range-thumb{border:none;height:16px;width:16px;border-radius:50%;background:#385380}input.vis-configuration.vis-config-range:-moz-focusring{outline:1px solid #fff;outline-offset:-1px}input.vis-configuration.vis-config-range::-ms-track{width:300px;height:5px;background:transparent;border-color:transparent;border-width:6px 0;color:transparent}input.vis-configuration.vis-config-range::-ms-fill-lower{background:#777;border-radius:10px}input.vis-configuration.vis-config-range::-ms-fill-upper{background:#ddd;border-radius:10px}input.vis-configuration.vis-config-range::-ms-thumb{border:none;height:16px;width:16px;border-radius:50%;background:#385380}input.vis-configuration.vis-config-range:focus::-ms-fill-lower{background:#888}input.vis-configuration.vis-config-range:focus::-ms-fill-upper{background:#ccc}.vis-configuration-popup{position:absolute;background:rgba(57,76,89,.85);border:2px solid #f2faff;line-height:30px;height:30px;width:150px;text-align:center;color:#fff;font-size:14px;border-radius:4px;-webkit-transition:opacity .3s ease-in-out;-moz-transition:opacity .3s ease-in-out;transition:opacity .3s ease-in-out}.vis-configuration-popup:after,.vis-configuration-popup:before{left:100%;top:50%;border:solid transparent;content:" ";height:0;width:0;position:absolute;pointer-events:none}.vis-configuration-popup:after{border-color:rgba(136,183,213,0) rgba(136,183,213,0) rgba(136,183,213,0) rgba(57,76,89,.85);border-width:8px;margin-top:-8px}.vis-configuration-popup:before{border-color:rgba(194,225,245,0) rgba(194,225,245,0) rgba(194,225,245,0) #f2faff;border-width:12px;margin-top:-12px}div.vis-tooltip{position:absolute;visibility:hidden;padding:5px;white-space:nowrap;font-family:verdana;font-size:14px;color:#000;background-color:#f5f4ed;-moz-border-radius:3px;-webkit-border-radius:3px;border-radius:3px;border:1px solid #808074;box-shadow:3px 3px 10px rgba(0,0,0,.2);pointer-events:none;z-index:5}.vis-panel{position:absolute;padding:0;margin:0;box-sizing:border-box}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-left,.vis-panel.vis-right,.vis-panel.vis-top{border:1px #bfbfbf}.vis-panel.vis-center,.vis-panel.vis-left,.vis-panel.vis-right{border-top-style:solid;border-bottom-style:solid;overflow:hidden}.vis-left.vis-panel.vis-vertical-scroll,.vis-right.vis-panel.vis-vertical-scroll{height:100%;overflow-x:hidden;overflow-y:scroll}.vis-left.vis-panel.vis-vertical-scroll{direction:rtl}.vis-left.vis-panel.vis-vertical-scroll .vis-content,.vis-right.vis-panel.vis-vertical-scroll{direction:ltr}.vis-right.vis-panel.vis-vertical-scroll .vis-content{direction:rtl}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-top{border-left-style:solid;border-right-style:solid}.vis-background{overflow:hidden}.vis-panel>.vis-content{position:relative}.vis-panel .vis-shadow{position:absolute;width:100%;height:1px;box-shadow:0 0 10px rgba(0,0,0,.8)}.vis-panel .vis-shadow.vis-top{top:-1px;left:0}.vis-panel .vis-shadow.vis-bottom{bottom:-1px;left:0}.vis-current-time{background-color:#ff7f6e;width:2px;z-index:1;pointer-events:none}.vis-rolling-mode-btn{height:40px;width:40px;position:absolute;top:7px;right:20px;border-radius:50%;font-size:28px;cursor:pointer;opacity:.8;color:#fff;font-weight:700;text-align:center;background:#3876c2}.vis-rolling-mode-btn:before{content:"\26F6"}.vis-rolling-mode-btn:hover{opacity:1}.vis-custom-time{background-color:#6e94ff;width:2px;cursor:move;z-index:1}.vis-custom-time>.vis-custom-time-marker{background-color:inherit;color:#fff;font-size:12px;white-space:nowrap;padding:3px 5px;top:0;cursor:auto;z-index:inherit}.vis-timeline{position:relative;border:1px solid #bfbfbf;overflow:hidden;padding:0;margin:0;box-sizing:border-box}.vis-loading-screen{width:100%;height:100%;position:absolute;top:0;left:0}.vis-graph-group0{fill:#4f81bd;fill-opacity:0;stroke-width:2px;stroke:#4f81bd}.vis-graph-group1{fill:#f79646;fill-opacity:0;stroke-width:2px;stroke:#f79646}.vis-graph-group2{fill:#8c51cf;fill-opacity:0;stroke-width:2px;stroke:#8c51cf}.vis-graph-group3{fill:#75c841;fill-opacity:0;stroke-width:2px;stroke:#75c841}.vis-graph-group4{fill:#ff0100;fill-opacity:0;stroke-width:2px;stroke:#ff0100}.vis-graph-group5{fill:#37d8e6;fill-opacity:0;stroke-width:2px;stroke:#37d8e6}.vis-graph-group6{fill:#042662;fill-opacity:0;stroke-width:2px;stroke:#042662}.vis-graph-group7{fill:#00ff26;fill-opacity:0;stroke-width:2px;stroke:#00ff26}.vis-graph-group8{fill:#f0f;fill-opacity:0;stroke-width:2px;stroke:#f0f}.vis-graph-group9{fill:#8f3938;fill-opacity:0;stroke-width:2px;stroke:#8f3938}.vis-timeline .vis-fill{fill-opacity:.1;stroke:none}.vis-timeline .vis-bar{fill-opacity:.5;stroke-width:1px}.vis-timeline .vis-point{stroke-width:2px;fill-opacity:1}.vis-timeline .vis-legend-background{stroke-width:1px;fill-opacity:.9;fill:#fff;stroke:#c2c2c2}.vis-timeline .vis-outline{stroke-width:1px;fill-opacity:1;fill:#fff;stroke:#e5e5e5}.vis-timeline .vis-icon-fill{fill-opacity:.3;stroke:none}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-horizontal{position:absolute;width:100%;height:0;border-bottom:1px solid}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-minor{border-color:#e5e5e5}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-major{border-color:#bfbfbf}.vis-data-axis .vis-y-axis.vis-major{width:100%;position:absolute;color:#4d4d4d;white-space:nowrap}.vis-data-axis .vis-y-axis.vis-major.vis-measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-minor{position:absolute;width:100%;color:#bebebe;white-space:nowrap}.vis-data-axis .vis-y-axis.vis-minor.vis-measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-title{position:absolute;color:#4d4d4d;white-space:nowrap;bottom:20px;text-align:center}.vis-data-axis .vis-y-axis.vis-title.vis-measure{padding:0;margin:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-title.vis-left{bottom:0;-webkit-transform-origin:left top;-moz-transform-origin:left top;-ms-transform-origin:left top;-o-transform-origin:left top;transform-origin:left bottom;-webkit-transform:rotate(-90deg);-moz-transform:rotate(-90deg);-ms-transform:rotate(-90deg);-o-transform:rotate(-90deg);transform:rotate(-90deg)}.vis-data-axis .vis-y-axis.vis-title.vis-right{bottom:0;-webkit-transform-origin:right bottom;-moz-transform-origin:right bottom;-ms-transform-origin:right bottom;-o-transform-origin:right bottom;transform-origin:right bottom;-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.vis-legend{background-color:rgba(247,252,255,.65);padding:5px;border:1px solid #b3b3b3;box-shadow:2px 2px 10px hsla(0,0%,60.4%,.55)}.vis-legend-text{white-space:nowrap;display:inline-block}.vis-itemset{position:relative;padding:0;margin:0;box-sizing:border-box}.vis-itemset .vis-background,.vis-itemset .vis-foreground{position:absolute;width:100%;height:100%;overflow:visible}.vis-axis{position:absolute;width:100%;height:0;left:0;z-index:1}.vis-foreground .vis-group{position:relative;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis-foreground .vis-group:last-child{border-bottom:none}.vis-nesting-group{cursor:pointer}.vis-label.vis-nested-group.vis-group-level-unknown-but-gte1{background:#f5f5f5}.vis-label.vis-nested-group.vis-group-level-0{background-color:#fff}.vis-ltr .vis-label.vis-nested-group.vis-group-level-0 .vis-inner{padding-left:0}.vis-rtl .vis-label.vis-nested-group.vis-group-level-0 .vis-inner{padding-right:0}.vis-label.vis-nested-group.vis-group-level-1{background-color:rgba(0,0,0,.05)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-1 .vis-inner{padding-left:15px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-1 .vis-inner{padding-right:15px}.vis-label.vis-nested-group.vis-group-level-2{background-color:rgba(0,0,0,.1)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-2 .vis-inner{padding-left:30px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-2 .vis-inner{padding-right:30px}.vis-label.vis-nested-group.vis-group-level-3{background-color:rgba(0,0,0,.15)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-3 .vis-inner{padding-left:45px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-3 .vis-inner{padding-right:45px}.vis-label.vis-nested-group.vis-group-level-4{background-color:rgba(0,0,0,.2)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-4 .vis-inner{padding-left:60px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-4 .vis-inner{padding-right:60px}.vis-label.vis-nested-group.vis-group-level-5{background-color:rgba(0,0,0,.25)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-5 .vis-inner{padding-left:75px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-5 .vis-inner{padding-right:75px}.vis-label.vis-nested-group.vis-group-level-6{background-color:rgba(0,0,0,.3)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-6 .vis-inner{padding-left:90px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-6 .vis-inner{padding-right:90px}.vis-label.vis-nested-group.vis-group-level-7{background-color:rgba(0,0,0,.35)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-7 .vis-inner{padding-left:105px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-7 .vis-inner{padding-right:105px}.vis-label.vis-nested-group.vis-group-level-8{background-color:rgba(0,0,0,.4)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-8 .vis-inner{padding-left:120px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-8 .vis-inner{padding-right:120px}.vis-label.vis-nested-group.vis-group-level-9{background-color:rgba(0,0,0,.45)}.vis-ltr .vis-label.vis-nested-group.vis-group-level-9 .vis-inner{padding-left:135px}.vis-rtl .vis-label.vis-nested-group.vis-group-level-9 .vis-inner{padding-right:135px}.vis-label.vis-nested-group{background-color:rgba(0,0,0,.5)}.vis-ltr .vis-label.vis-nested-group .vis-inner{padding-left:150px}.vis-rtl .vis-label.vis-nested-group .vis-inner{padding-right:150px}.vis-group-level-unknown-but-gte1{border:1px solid red}.vis-label.vis-nesting-group:before{display:inline-block;width:15px}.vis-label.vis-nesting-group.expanded:before{content:"\25BC"}.vis-label.vis-nesting-group.collapsed:before{content:"\25B6"}.vis-rtl .vis-label.vis-nesting-group.collapsed:before{content:"\25C0"}.vis-ltr .vis-label:not(.vis-nesting-group):not(.vis-group-level-0){padding-left:15px}.vis-rtl .vis-label:not(.vis-nesting-group):not(.vis-group-level-0){padding-right:15px}.vis-overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-labelset{overflow:hidden}.vis-labelset,.vis-labelset .vis-label{position:relative;box-sizing:border-box}.vis-labelset .vis-label{left:0;top:0;width:100%;color:#4d4d4d;border-bottom:1px solid #bfbfbf}.vis-labelset .vis-label.draggable{cursor:pointer}.vis-group-is-dragging{background:rgba(0,0,0,.1)}.vis-labelset .vis-label:last-child{border-bottom:none}.vis-labelset .vis-label .vis-inner{display:inline-block;padding:5px}.vis-labelset .vis-label .vis-inner.vis-hidden{padding:0}.vis-time-axis{position:relative;overflow:hidden}.vis-time-axis.vis-foreground{top:0;left:0;width:100%}.vis-time-axis.vis-background{position:absolute;top:0;left:0;width:100%;height:100%}.vis-time-axis .vis-text{position:absolute;color:#4d4d4d;padding:3px;overflow:hidden;box-sizing:border-box;white-space:nowrap}.vis-time-axis .vis-text.vis-measure{position:absolute;padding-left:0;padding-right:0;margin-left:0;margin-right:0;visibility:hidden}.vis-time-axis .vis-grid.vis-vertical{position:absolute;border-left:1px solid}.vis-time-axis .vis-grid.vis-vertical-rtl{position:absolute;border-right:1px solid}.vis-time-axis .vis-grid.vis-minor{border-color:#e5e5e5}.vis-time-axis .vis-grid.vis-major{border-color:#bfbfbf}.vis-item{position:absolute;color:#1a1a1a;border-color:#97b0f8;border-width:1px;background-color:#d5ddf6;display:inline-block;z-index:1}.vis-item.vis-selected{border-color:#ffc200;background-color:#fff785;z-index:2}.vis-editable.vis-selected{cursor:move}.vis-item.vis-point.vis-selected{background-color:#fff785}.vis-item.vis-box{text-align:center;border-style:solid;border-radius:2px}.vis-item.vis-point{background:none}.vis-item.vis-dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px}.vis-item.vis-range{border-style:solid;border-radius:2px;box-sizing:border-box}.vis-item.vis-background{border:none;background-color:rgba(213,221,246,.4);box-sizing:border-box;padding:0;margin:0}.vis-item .vis-item-overflow{position:relative;width:100%;height:100%;padding:0;margin:0;overflow:hidden}.vis-item-visible-frame{white-space:nowrap}.vis-item.vis-range .vis-item-content{position:relative;display:inline-block}.vis-item.vis-background .vis-item-content{position:absolute;display:inline-block}.vis-item.vis-line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis-item .vis-item-content{white-space:nowrap;box-sizing:border-box;padding:5px}.vis-item .vis-onUpdateTime-tooltip{position:absolute;background:#4f81bd;color:#fff;width:200px;text-align:center;white-space:nowrap;padding:5px;border-radius:1px;transition:.4s;-o-transition:.4s;-moz-transition:.4s;-webkit-transition:.4s}.vis-item .vis-delete,.vis-item .vis-delete-rtl{position:absolute;top:0;width:24px;height:24px;box-sizing:border-box;padding:0 5px;cursor:pointer;-webkit-transition:background .2s linear;-moz-transition:background .2s linear;-ms-transition:background .2s linear;-o-transition:background .2s linear;transition:background .2s linear}.vis-item .vis-delete{right:-24px}.vis-item .vis-delete-rtl{left:-24px}.vis-item .vis-delete-rtl:after,.vis-item .vis-delete:after{content:"\00D7";color:red;font-family:arial,sans-serif;font-size:22px;font-weight:700;-webkit-transition:color .2s linear;-moz-transition:color .2s linear;-ms-transition:color .2s linear;-o-transition:color .2s linear;transition:color .2s linear}.vis-item .vis-delete-rtl:hover,.vis-item .vis-delete:hover{background:red}.vis-item .vis-delete-rtl:hover:after,.vis-item .vis-delete:hover:after{color:#fff}.vis-item .vis-drag-center{position:absolute;width:100%;height:100%;top:0;left:0;cursor:move}.vis-item.vis-range .vis-drag-left{left:-4px;cursor:w-resize}.vis-item.vis-range .vis-drag-left,.vis-item.vis-range .vis-drag-right{position:absolute;width:24px;max-width:20%;min-width:2px;height:100%;top:0}.vis-item.vis-range .vis-drag-right{right:-4px;cursor:e-resize}.vis-range.vis-item.vis-readonly .vis-drag-left,.vis-range.vis-item.vis-readonly .vis-drag-right{cursor:auto}.vis-item.vis-cluster{vertical-align:center;text-align:center;border-style:solid;border-radius:2px}.vis-item.vis-cluster-line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis-item.vis-cluster-dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px} +/*# sourceMappingURL=vis-timeline-graph2d.min.css.map */ \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css.map b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css.map new file mode 100644 index 0000000000000..3c165a792d0f3 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.css.map @@ -0,0 +1 @@ +{"version":3,"sources":["bootstrap.css","activator.css","configuration.css","tooltip.css","panel.css","currenttime.css","customtime.css","timeline.css","pathStyles.css","dataaxis.css","itemset.css","labelset.css","timeaxis.css","item.css"],"names":[],"mappings":"AAEA,mBACE,YAAa,CACb,UACF,CCLA,cACE,iBAAkB,CAClB,KAAM,CACN,MAAO,CACP,UAAW,CACX,WAAY,CAGZ,UACF,CAEA,YACE,2BACF,CCbA,sBACI,iBAAiB,CACjB,aAAa,CACb,UAAU,CACV,cACJ,CAEA,8BACI,aAAa,CACb,WACJ,CAEA,oCACE,UAAW,CACX,UAAW,CACX,aACF,CAEA,kDACI,aAAa,CACb,WAAW,CACX,qBAAyB,CACzB,wBAAwB,CACxB,iBAAiB,CACjB,eAAe,CACf,SAAS,CACT,gBACJ,CAEA,wCACI,aAAa,CACb,WAAW,CACX,WAAW,CACX,qBAAsB,CACtB,gBAAgB,CAChB,wBAAyB,CACzB,wBAAwB,CACxB,iBAAiB,CACjB,eAAe,CACf,SAAS,CACT,gBAAgB,CAChB,cAAe,CACf,kBACJ,CAEA,8CACI,wBAAyB,CACzB,wBAAwB,CACxB,UACJ,CAEA,sCACI,aAAa,CACb,UAAU,CACV,WAAW,CACX,WAAW,CACX,qBAAsB,CACtB,gBACJ,CAGA,oDACI,SAAS,CACT,wBAAyB,CACzB,gBAAgB,CAChB,iBACJ,CACA,oDACI,SAAS,CACT,wBAAyB,CACzB,gBAAgB,CAChB,iBACJ,CACA,oDACI,SAAS,CACT,wBAAyB,CACzB,gBAAgB,CAChB,iBACJ,CAEA,wCACI,cAAc,CACd,eACJ,CAEA,uCACI,WAAW,CACX,WAAW,CACX,gBACJ,CAEA,qDACI,WACJ,CACA,qDACI,WACJ,CAEA,4CACI,OAAO,CACP,UAAU,CACV,WAAW,CACX,qBAAwB,CACxB,iBAAiB,CACjB,SAAW,CACX,QAAU,CACV,cACJ,CAEA,4CACI,SACJ,CAGA,8CACI,iBAAiB,CACjB,QAAQ,CACR,UAAU,CAEV,WAAW,CACX,QAAQ,CACR,mBACJ,CAEA,yCAEI,uBAAwB,CAGxB,mBAAuB,CACvB,4BAA8B,CAG9B,WAAY,CACZ,WACJ,CACA,wEACI,WAAY,CACZ,UAAW,CACX,kBAAmB,CACnB,0DAA+D,CAC/D,sGAA4G,CAC5G,2DAAiE,CACjE,wDAA4D,CAC5D,yDAA6D,CAC7D,wDAA+D,CAC/D,+GAAmH,CAEnH,qBAAyB,CACzB,yBAAmC,CACnC,iBACJ,CACA,+DACI,uBAAwB,CACxB,wBAAyB,CACzB,WAAY,CACZ,UAAW,CACX,iBAAkB,CAClB,kBAAmB,CACnB,2DAAgE,CAChE,uGAA6G,CAC7G,uDAAkE,CAClE,yDAA6D,CAC7D,0DAA8D,CAC9D,oDAAgE,CAChE,+GAAmH,CACnH,4BAAmC,CACnC,eACJ,CACA,+CACI,YACJ,CACA,8EACI,kBAAmB,CACnB,0DAA8D,CAC9D,sGAA4G,CAC5G,2DAAiE,CACjE,wDAA4D,CAC5D,yDAA6D,CAC7D,wDAA+D,CAC/D,+GACJ,CAEA,2DACI,WAAY,CACZ,WAAY,CACZ,kBAAmB,CACnB,0DAA+D,CAC/D,sGAA4G,CAC5G,2DAAiE,CACjE,wDAA4D,CAC5D,yDAA6D,CAC7D,wDAA+D,CAC/D,+GAAmH,CAEnH,qBAAyB,CACzB,yBAAmC,CACnC,iBACJ,CACA,2DACI,WAAY,CACZ,WAAY,CACZ,UAAW,CAEX,iBAAkB,CAClB,kBACJ,CAGA,wDACI,sBAAwB,CACxB,mBACJ,CAEA,oDACI,WAAY,CACZ,UAAW,CAGX,sBAAuB,CAGvB,wBAAyB,CACzB,kBAAmB,CAGnB,iBACJ,CACA,yDACI,eAAgB,CAChB,kBACJ,CACA,yDACI,eAAgB,CAChB,kBACJ,CACA,oDACI,WAAY,CACZ,WAAY,CACZ,UAAW,CACX,iBAAkB,CAClB,kBACJ,CACA,+DACI,eACJ,CACA,+DACI,eACJ,CAEA,yBACI,iBAAkB,CAClB,6BAAkC,CAClC,wBAAyB,CACzB,gBAAgB,CAChB,WAAW,CACX,WAAW,CACX,iBAAiB,CACjB,UAAc,CACd,cAAc,CACd,iBAAiB,CACjB,0CAA4C,CAC5C,uCAAyC,CACzC,kCACJ,CACA,+DACI,SAAU,CACV,OAAQ,CACR,wBAAyB,CACzB,WAAY,CACZ,QAAS,CACT,OAAQ,CACR,iBAAkB,CAClB,mBACJ,CAEA,+BAEI,2FAAyC,CACzC,gBAAiB,CACjB,eACJ,CACA,gCAEI,gFAA0B,CAC1B,iBAAkB,CAClB,gBACJ,CC/RA,gBACE,iBAAkB,CAClB,iBAAkB,CAClB,WAAY,CACZ,kBAAmB,CAEnB,mBAAoB,CACpB,cAAc,CACd,UAAa,CACb,wBAAyB,CAEzB,sBAAuB,CACvB,yBAA0B,CAC1B,iBAAkB,CAClB,wBAAyB,CAEzB,sCAA2C,CAC3C,mBAAoB,CAEpB,SACF,CCpBA,WACE,iBAAkB,CAElB,SAAU,CACV,QAAS,CAET,qBACF,CAEA,wGAKE,kBACF,CAEA,+DAGE,sBAAuB,CACvB,yBAA0B,CAC1B,eACF,CAEA,iFACE,WAAY,CACZ,iBAAkB,CAClB,iBACF,CAEA,wCACE,aACF,CAMA,8FACE,aACF,CAEA,sDACE,aACF,CAEA,+DAGE,uBAAwB,CACxB,wBACF,CAEA,gBACE,eACF,CAEA,wBACE,iBACF,CAEA,uBACE,iBAAkB,CAClB,UAAW,CACX,UAAW,CACX,kCAIF,CAEA,+BACE,QAAS,CACT,MACF,CAEA,kCACE,WAAY,CACZ,MACF,CChFA,kBACE,wBAAyB,CACzB,SAAU,CACV,SAAU,CACV,mBACF,CAEA,sBACE,WAAY,CACZ,UAAW,CACX,iBAAkB,CAClB,OAAQ,CACR,UAAW,CACX,iBAAkB,CAClB,cAAe,CACf,cAAe,CACf,UAAY,CACZ,UAAY,CACZ,eAAiB,CACjB,iBAAkB,CAClB,kBACF,CACA,6BACE,eACF,CAEA,4BACE,SACF,CC5BA,iBACE,wBAAyB,CACzB,SAAU,CACV,WAAY,CACZ,SACF,CAEA,yCACE,wBAAyB,CACzB,UAAY,CACZ,cAAe,CACf,kBAAmB,CACnB,eAAgB,CAChB,KAAQ,CACR,WAAe,CACf,eACF,CCfA,cACE,iBAAkB,CAClB,wBAAyB,CACzB,eAAgB,CAChB,SAAU,CACV,QAAS,CACT,qBACF,CAEA,oBACE,UAAW,CACX,WAAY,CACZ,iBAAkB,CAClB,KAAM,CACN,MACF,CChBA,kBACI,YAAY,CACZ,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAY,CACZ,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,YAAY,CACZ,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,kBACI,SAAY,CACZ,cAAc,CACd,gBAAgB,CAChB,WACJ,CAEA,kBACI,YAAa,CACb,cAAc,CACd,gBAAgB,CAChB,cACJ,CAEA,wBACI,eAAgB,CAChB,WACJ,CAGA,uBACI,eAAgB,CAChB,gBACJ,CAEA,yBACI,gBAAgB,CAChB,cACJ,CAGA,qCACI,gBAAgB,CAChB,eAAgB,CAChB,SAAa,CACb,cACJ,CAGA,2BACI,gBAAgB,CAChB,cAAc,CACd,SAAa,CACb,cACJ,CAEA,6BACI,eAAgB,CAChB,WACJ,CCxGA,kEACE,iBAAkB,CAClB,UAAW,CACX,QAAS,CACT,uBACF,CAEA,6DACE,oBACF,CAEA,6DACE,oBACF,CAGA,qCACE,UAAW,CACX,iBAAkB,CAClB,aAAc,CACd,kBACF,CAEA,iDACE,SAAU,CACV,QAAS,CACT,QAAS,CACT,iBAAkB,CAClB,UACF,CAGA,qCACE,iBAAkB,CAClB,UAAW,CACX,aAAc,CACd,kBACF,CAEA,iDACE,SAAU,CACV,QAAS,CACT,QAAS,CACT,iBAAkB,CAClB,UACF,CAEA,qCACE,iBAAkB,CAClB,aAAc,CACd,kBAAmB,CACnB,WAAY,CACZ,iBACF,CAEA,iDACE,SAAU,CACV,QAAS,CACT,iBAAkB,CAClB,UACF,CAEA,8CACE,QAAS,CACT,iCAAkC,CAClC,8BAA+B,CAC/B,6BAA8B,CAC9B,4BAA6B,CAC7B,4BAA6B,CAC7B,gCAAiC,CACjC,6BAA8B,CAC9B,4BAA6B,CAC7B,2BAA4B,CAC5B,wBACF,CAEA,+CACE,QAAS,CACT,qCAAsC,CACtC,kCAAmC,CACnC,iCAAkC,CAClC,gCAAiC,CACjC,6BAA8B,CAC9B,+BAAgC,CAChC,4BAA6B,CAC7B,2BAA4B,CAC5B,0BAA2B,CAC3B,uBACF,CAEA,YACE,sCAA2C,CAC3C,WAAY,CACZ,wBAAyB,CACzB,4CACF,CAEA,iBAEE,kBAAmB,CACnB,oBACF,CCrGA,aACE,iBAAkB,CAClB,SAAU,CACV,QAAS,CAET,qBACF,CAEA,0DAEE,iBAAkB,CAClB,UAAW,CACX,WAAY,CACZ,gBACF,CAEA,UACE,iBAAkB,CAClB,UAAW,CACX,QAAS,CACT,MAAO,CACP,SACF,CAEA,2BACE,iBAAkB,CAClB,qBAAsB,CACtB,+BACF,CAEA,sCACE,kBACF,CAEA,mBACE,cACF,CAEA,6DACE,kBACF,CACA,8CACE,qBACF,CACA,kEACE,cACF,CACA,kEACE,eACF,CACA,8CACE,gCACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,+BACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,gCACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,+BACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,gCACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,+BACF,CACA,kEACE,iBACF,CACA,kEACE,kBACF,CACA,8CACE,gCACF,CACA,kEACE,kBACF,CACA,kEACE,mBACF,CACA,8CACE,+BACF,CACA,kEACE,kBACF,CACA,kEACE,mBACF,CACA,8CACE,gCACF,CACA,kEACE,kBACF,CACA,kEACE,mBACF,CAGA,4BACE,+BACF,CACA,gDACE,kBACF,CACA,gDACE,mBACF,CAEA,kCACE,oBACF,CAGA,oCAEE,oBAAqB,CACrB,UACF,CACA,6CACE,eACF,CACA,8CACE,eACF,CACA,uDACE,eACF,CAEA,oEACE,iBACF,CACA,oEACE,kBACF,CAEA,aACE,iBAAkB,CAClB,KAAM,CACN,MAAO,CACP,UAAW,CACX,WAAY,CACZ,UACF,CCjLA,cAGE,eAGF,CAEA,uCAPE,iBAAkB,CAIlB,qBAWF,CARA,yBAEE,MAAO,CACP,KAAM,CACN,UAAW,CACX,aAAc,CAMd,+BAHF,CAMA,mCACE,cACF,CAEA,uBACE,yBACF,CAEA,oCACE,kBACF,CAEA,oCACE,oBAAqB,CACrB,WACF,CAEA,+CACE,SACF,CC1CA,eACE,iBAAkB,CAClB,eACF,CAEA,8BACE,KAAM,CACN,MAAO,CACP,UACF,CAEA,8BACE,iBAAkB,CAClB,KAAM,CACN,MAAO,CACP,UAAW,CACX,WACF,CAEA,yBACE,iBAAkB,CAClB,aAAc,CACd,WAAY,CACZ,eAAgB,CAChB,qBAAsB,CAEtB,kBACF,CAEA,qCACE,iBAAkB,CAClB,cAAe,CACf,eAAgB,CAChB,aAAc,CACd,cAAe,CACf,iBACF,CAEA,sCACE,iBAAkB,CAClB,qBACF,CAEA,0CACE,iBAAkB,CAClB,sBACF,CAEA,mCACE,oBACF,CAEA,mCACE,oBACF,CCrDA,UACE,iBAAkB,CAClB,aAAc,CACd,oBAAqB,CACrB,gBAAiB,CACjB,wBAAyB,CACzB,oBAAqB,CACrB,SAEF,CAEA,uBACE,oBAAqB,CACrB,wBAAyB,CAGzB,SACF,CAEA,2BACE,WACF,CAEA,iCACE,wBACF,CAEA,kBACE,iBAAkB,CAClB,kBAAmB,CACnB,iBACF,CAEA,oBACE,eACF,CAEA,kBACE,iBAAkB,CAClB,SAAU,CACV,gBAAiB,CACjB,kBAAmB,CACnB,iBACF,CAEA,oBACE,kBAAmB,CACnB,iBAAkB,CAClB,qBACF,CAEA,yBACE,WAAY,CACZ,qCAA0C,CAC1C,qBAAsB,CACtB,SAAU,CACV,QACF,CAEA,6BACE,iBAAkB,CAClB,UAAW,CACX,WAAY,CACZ,SAAU,CACV,QAAS,CACT,eACF,CAEA,wBACE,kBACF,CAEA,sCACE,iBAAkB,CAClB,oBACF,CAEA,2CACE,iBAAkB,CAClB,oBACF,CAEA,mBACE,SAAU,CACV,iBAAkB,CAClB,OAAQ,CACR,qBAAsB,CACtB,uBACF,CAEA,4BACE,kBAAmB,CACnB,qBAAsB,CACtB,WACF,CAEA,oCACE,iBAAkB,CAClB,kBAAmB,CACnB,UAAY,CACZ,WAAY,CACZ,iBAAkB,CAClB,kBAAmB,CACnB,WAAY,CACZ,iBAAkB,CAClB,cAAgB,CAChB,iBAAmB,CACnB,mBAAqB,CACrB,sBACF,CAEA,gDACE,iBAAkB,CAClB,KAAQ,CACR,UAAW,CACX,WAAY,CACZ,qBAAsB,CACtB,aAAgB,CAChB,cAAe,CAEf,wCAA0C,CAC1C,qCAAuC,CACvC,oCAAsC,CACtC,mCAAqC,CACrC,gCACF,CAEA,sBACE,WACF,CAEA,0BACE,UACF,CAEA,4DACE,eAAgB,CAChB,SAAU,CACV,4BAA8B,CAC9B,cAAe,CACf,eAAiB,CAEjB,mCAAqC,CACrC,gCAAkC,CAClC,+BAAiC,CACjC,8BAAgC,CAChC,2BACF,CAEA,4DACE,cACF,CAEA,wEACE,UACF,CAEA,2BACE,iBAAkB,CAClB,UAAW,CACX,WAAY,CACZ,KAAM,CACN,MAAS,CACT,WACF,CAEA,mCAOE,SAAU,CAEV,eACF,CAEA,uEAXE,iBAAkB,CAClB,UAAW,CACX,aAAc,CACd,aAAc,CACd,WAAY,CACZ,KAgBF,CAVA,oCAOE,UAAW,CAEX,eACF,CAEA,iGAEE,WACF,CAEA,sBACE,qBAAsB,CACtB,iBAAkB,CAClB,kBAAmB,CACnB,iBACF,CAEA,2BACE,SAAU,CACV,iBAAkB,CAClB,OAAQ,CACR,qBAAsB,CACtB,uBACF,CAEA,0BACE,iBAAkB,CAClB,SAAU,CACV,gBAAiB,CACjB,kBAAmB,CACnB,iBACF","file":"vis-timeline-graph2d.min.css","sourcesContent":["/* override some bootstrap styles screwing up the timelines css */\n\n.vis [class*=\"span\"] {\n min-height: 0;\n width: auto;\n}\n",".vis .overlay {\n position: absolute;\n top: 0;\n left: 0;\n width: 100%;\n height: 100%;\n\n /* Must be displayed above for example selected Timeline items */\n z-index: 10;\n}\n\n.vis-active {\n box-shadow: 0 0 10px #86d5f8;\n}\n","div.vis-configuration {\n position:relative;\n display:block;\n float:left;\n font-size:12px;\n}\n\ndiv.vis-configuration-wrapper {\n display:block;\n width:700px;\n}\n\ndiv.vis-configuration-wrapper::after {\n clear: both;\n content: \"\";\n display: block;\n}\n\ndiv.vis-configuration.vis-config-option-container{\n display:block;\n width:495px;\n background-color: #ffffff;\n border:2px solid #f7f8fa;\n border-radius:4px;\n margin-top:20px;\n left:10px;\n padding-left:5px;\n}\n\ndiv.vis-configuration.vis-config-button{\n display:block;\n width:495px;\n height:25px;\n vertical-align: middle;\n line-height:25px;\n background-color: #f7f8fa;\n border:2px solid #ceced0;\n border-radius:4px;\n margin-top:20px;\n left:10px;\n padding-left:5px;\n cursor: pointer;\n margin-bottom:30px;\n}\n\ndiv.vis-configuration.vis-config-button.hover{\n background-color: #4588e6;\n border:2px solid #214373;\n color:#ffffff;\n}\n\ndiv.vis-configuration.vis-config-item{\n display:block;\n float:left;\n width:495px;\n height:25px;\n vertical-align: middle;\n line-height:25px;\n}\n\n\ndiv.vis-configuration.vis-config-item.vis-config-s2{\n left:10px;\n background-color: #f7f8fa;\n padding-left:5px;\n border-radius:3px;\n}\ndiv.vis-configuration.vis-config-item.vis-config-s3{\n left:20px;\n background-color: #e4e9f0;\n padding-left:5px;\n border-radius:3px;\n}\ndiv.vis-configuration.vis-config-item.vis-config-s4{\n left:30px;\n background-color: #cfd8e6;\n padding-left:5px;\n border-radius:3px;\n}\n\ndiv.vis-configuration.vis-config-header{\n font-size:18px;\n font-weight: bold;\n}\n\ndiv.vis-configuration.vis-config-label{\n width:120px;\n height:25px;\n line-height: 25px;\n}\n\ndiv.vis-configuration.vis-config-label.vis-config-s3{\n width:110px;\n}\ndiv.vis-configuration.vis-config-label.vis-config-s4{\n width:100px;\n}\n\ndiv.vis-configuration.vis-config-colorBlock{\n top:1px;\n width:30px;\n height:19px;\n border:1px solid #444444;\n border-radius:2px;\n padding:0px;\n margin:0px;\n cursor:pointer;\n}\n\ninput.vis-configuration.vis-config-checkbox {\n left:-5px;\n}\n\n\ninput.vis-configuration.vis-config-rangeinput{\n position:relative;\n top:-5px;\n width:60px;\n /*height:13px;*/\n padding:1px;\n margin:0;\n pointer-events:none;\n}\n\ninput.vis-configuration.vis-config-range{\n /*removes default webkit styles*/\n -webkit-appearance: none;\n\n /*fix for FF unable to apply focus style bug */\n border: 0px solid white;\n background-color:rgba(0,0,0,0);\n\n /*required for proper track sizing in FF*/\n width: 300px;\n height:20px;\n}\ninput.vis-configuration.vis-config-range::-webkit-slider-runnable-track {\n width: 300px;\n height: 5px;\n background: #dedede; /* Old browsers */\n background: -moz-linear-gradient(top, #dedede 0%, #c8c8c8 99%); /* FF3.6+ */\n background: -webkit-gradient(linear, left top, left bottom, color-stop(0%,#dedede), color-stop(99%,#c8c8c8)); /* Chrome,Safari4+ */\n background: -webkit-linear-gradient(top, #dedede 0%,#c8c8c8 99%); /* Chrome10+,Safari5.1+ */\n background: -o-linear-gradient(top, #dedede 0%, #c8c8c8 99%); /* Opera 11.10+ */\n background: -ms-linear-gradient(top, #dedede 0%,#c8c8c8 99%); /* IE10+ */\n background: linear-gradient(to bottom, #dedede 0%,#c8c8c8 99%); /* W3C */\n filter: progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8',GradientType=0 ); /* IE6-9 */\n\n border: 1px solid #999999;\n box-shadow: #aaaaaa 0px 0px 3px 0px;\n border-radius: 3px;\n}\ninput.vis-configuration.vis-config-range::-webkit-slider-thumb {\n -webkit-appearance: none;\n border: 1px solid #14334b;\n height: 17px;\n width: 17px;\n border-radius: 50%;\n background: #3876c2; /* Old browsers */\n background: -moz-linear-gradient(top, #3876c2 0%, #385380 100%); /* FF3.6+ */\n background: -webkit-gradient(linear, left top, left bottom, color-stop(0%,#3876c2), color-stop(100%,#385380)); /* Chrome,Safari4+ */\n background: -webkit-linear-gradient(top, #3876c2 0%,#385380 100%); /* Chrome10+,Safari5.1+ */\n background: -o-linear-gradient(top, #3876c2 0%,#385380 100%); /* Opera 11.10+ */\n background: -ms-linear-gradient(top, #3876c2 0%,#385380 100%); /* IE10+ */\n background: linear-gradient(to bottom, #3876c2 0%,#385380 100%); /* W3C */\n filter: progid:DXImageTransform.Microsoft.gradient( startColorstr='#3876c2', endColorstr='#385380',GradientType=0 ); /* IE6-9 */\n box-shadow: #111927 0px 0px 1px 0px;\n margin-top: -7px;\n}\ninput.vis-configuration.vis-config-range:focus {\n outline: none;\n}\ninput.vis-configuration.vis-config-range:focus::-webkit-slider-runnable-track {\n background: #9d9d9d; /* Old browsers */\n background: -moz-linear-gradient(top, #9d9d9d 0%, #c8c8c8 99%); /* FF3.6+ */\n background: -webkit-gradient(linear, left top, left bottom, color-stop(0%,#9d9d9d), color-stop(99%,#c8c8c8)); /* Chrome,Safari4+ */\n background: -webkit-linear-gradient(top, #9d9d9d 0%,#c8c8c8 99%); /* Chrome10+,Safari5.1+ */\n background: -o-linear-gradient(top, #9d9d9d 0%,#c8c8c8 99%); /* Opera 11.10+ */\n background: -ms-linear-gradient(top, #9d9d9d 0%,#c8c8c8 99%); /* IE10+ */\n background: linear-gradient(to bottom, #9d9d9d 0%,#c8c8c8 99%); /* W3C */\n filter: progid:DXImageTransform.Microsoft.gradient( startColorstr='#9d9d9d', endColorstr='#c8c8c8',GradientType=0 ); /* IE6-9 */\n}\n\ninput.vis-configuration.vis-config-range::-moz-range-track {\n width: 300px;\n height: 10px;\n background: #dedede; /* Old browsers */\n background: -moz-linear-gradient(top, #dedede 0%, #c8c8c8 99%); /* FF3.6+ */\n background: -webkit-gradient(linear, left top, left bottom, color-stop(0%,#dedede), color-stop(99%,#c8c8c8)); /* Chrome,Safari4+ */\n background: -webkit-linear-gradient(top, #dedede 0%,#c8c8c8 99%); /* Chrome10+,Safari5.1+ */\n background: -o-linear-gradient(top, #dedede 0%, #c8c8c8 99%); /* Opera 11.10+ */\n background: -ms-linear-gradient(top, #dedede 0%,#c8c8c8 99%); /* IE10+ */\n background: linear-gradient(to bottom, #dedede 0%,#c8c8c8 99%); /* W3C */\n filter: progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8',GradientType=0 ); /* IE6-9 */\n\n border: 1px solid #999999;\n box-shadow: #aaaaaa 0px 0px 3px 0px;\n border-radius: 3px;\n}\ninput.vis-configuration.vis-config-range::-moz-range-thumb {\n border: none;\n height: 16px;\n width: 16px;\n\n border-radius: 50%;\n background: #385380;\n}\n\n/*hide the outline behind the border*/\ninput.vis-configuration.vis-config-range:-moz-focusring{\n outline: 1px solid white;\n outline-offset: -1px;\n}\n\ninput.vis-configuration.vis-config-range::-ms-track {\n width: 300px;\n height: 5px;\n\n /*remove bg colour from the track, we'll use ms-fill-lower and ms-fill-upper instead */\n background: transparent;\n\n /*leave room for the larger thumb to overflow with a transparent border */\n border-color: transparent;\n border-width: 6px 0;\n\n /*remove default tick marks*/\n color: transparent;\n}\ninput.vis-configuration.vis-config-range::-ms-fill-lower {\n background: #777;\n border-radius: 10px;\n}\ninput.vis-configuration.vis-config-range::-ms-fill-upper {\n background: #ddd;\n border-radius: 10px;\n}\ninput.vis-configuration.vis-config-range::-ms-thumb {\n border: none;\n height: 16px;\n width: 16px;\n border-radius: 50%;\n background: #385380;\n}\ninput.vis-configuration.vis-config-range:focus::-ms-fill-lower {\n background: #888;\n}\ninput.vis-configuration.vis-config-range:focus::-ms-fill-upper {\n background: #ccc;\n}\n\n.vis-configuration-popup {\n position: absolute;\n background: rgba(57, 76, 89, 0.85);\n border: 2px solid #f2faff;\n line-height:30px;\n height:30px;\n width:150px;\n text-align:center;\n color: #ffffff;\n font-size:14px;\n border-radius:4px;\n -webkit-transition: opacity 0.3s ease-in-out;\n -moz-transition: opacity 0.3s ease-in-out;\n transition: opacity 0.3s ease-in-out;\n}\n.vis-configuration-popup:after, .vis-configuration-popup:before {\n left: 100%;\n top: 50%;\n border: solid transparent;\n content: \" \";\n height: 0;\n width: 0;\n position: absolute;\n pointer-events: none;\n}\n\n.vis-configuration-popup:after {\n border-color: rgba(136, 183, 213, 0);\n border-left-color: rgba(57, 76, 89, 0.85);\n border-width: 8px;\n margin-top: -8px;\n}\n.vis-configuration-popup:before {\n border-color: rgba(194, 225, 245, 0);\n border-left-color: #f2faff;\n border-width: 12px;\n margin-top: -12px;\n}","div.vis-tooltip {\n position: absolute;\n visibility: hidden;\n padding: 5px;\n white-space: nowrap;\n\n font-family: verdana;\n font-size:14px;\n color:#000000;\n background-color: #f5f4ed;\n\n -moz-border-radius: 3px;\n -webkit-border-radius: 3px;\n border-radius: 3px;\n border: 1px solid #808074;\n\n box-shadow: 3px 3px 10px rgba(0, 0, 0, 0.2);\n pointer-events: none;\n\n z-index: 5;\n}\n",".vis-panel {\n position: absolute;\n\n padding: 0;\n margin: 0;\n\n box-sizing: border-box;\n}\n\n.vis-panel.vis-center,\n.vis-panel.vis-left,\n.vis-panel.vis-right,\n.vis-panel.vis-top,\n.vis-panel.vis-bottom {\n border: 1px #bfbfbf;\n}\n\n.vis-panel.vis-center,\n.vis-panel.vis-left,\n.vis-panel.vis-right {\n border-top-style: solid;\n border-bottom-style: solid;\n overflow: hidden;\n}\n\n.vis-left.vis-panel.vis-vertical-scroll, .vis-right.vis-panel.vis-vertical-scroll {\n height: 100%;\n overflow-x: hidden;\n overflow-y: scroll;\n} \n\n.vis-left.vis-panel.vis-vertical-scroll {\n direction: rtl;\n}\n\n.vis-left.vis-panel.vis-vertical-scroll .vis-content {\n direction: ltr;\n}\n\n.vis-right.vis-panel.vis-vertical-scroll {\n direction: ltr;\n}\n\n.vis-right.vis-panel.vis-vertical-scroll .vis-content {\n direction: rtl;\n}\n\n.vis-panel.vis-center,\n.vis-panel.vis-top,\n.vis-panel.vis-bottom {\n border-left-style: solid;\n border-right-style: solid;\n}\n\n.vis-background {\n overflow: hidden;\n}\n\n.vis-panel > .vis-content {\n position: relative;\n}\n\n.vis-panel .vis-shadow {\n position: absolute;\n width: 100%;\n height: 1px;\n box-shadow: 0 0 10px rgba(0,0,0,0.8);\n /* TODO: find a nice way to ensure vis-shadows are drawn on top of items\n z-index: 1;\n */\n}\n\n.vis-panel .vis-shadow.vis-top {\n top: -1px;\n left: 0;\n}\n\n.vis-panel .vis-shadow.vis-bottom {\n bottom: -1px;\n left: 0;\n}",".vis-current-time {\n background-color: #FF7F6E;\n width: 2px;\n z-index: 1;\n pointer-events: none;\n}\n\n.vis-rolling-mode-btn {\n height: 40px;\n width: 40px;\n position: absolute;\n top: 7px;\n right: 20px;\n border-radius: 50%;\n font-size: 28px;\n cursor: pointer;\n opacity: 0.8;\n color: white;\n font-weight: bold;\n text-align: center;\n background: #3876c2;\n}\n.vis-rolling-mode-btn:before {\n content: \"\\26F6\";\n}\n\n.vis-rolling-mode-btn:hover {\n opacity: 1;\n}",".vis-custom-time {\n background-color: #6E94FF;\n width: 2px;\n cursor: move;\n z-index: 1;\n}\n\n.vis-custom-time > .vis-custom-time-marker {\n background-color: inherit;\n color: white;\n font-size: 12px;\n white-space: nowrap;\n padding: 3px 5px;\n top: 0px;\n cursor: initial;\n z-index: inherit;\n}","\n.vis-timeline {\n position: relative;\n border: 1px solid #bfbfbf;\n overflow: hidden;\n padding: 0;\n margin: 0;\n box-sizing: border-box;\n}\n\n.vis-loading-screen {\n width: 100%;\n height: 100%;\n position: absolute;\n top: 0;\n left: 0;\n}",".vis-graph-group0 {\n fill:#4f81bd;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #4f81bd;\n}\n\n.vis-graph-group1 {\n fill:#f79646;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #f79646;\n}\n\n.vis-graph-group2 {\n fill: #8c51cf;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #8c51cf;\n}\n\n.vis-graph-group3 {\n fill: #75c841;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #75c841;\n}\n\n.vis-graph-group4 {\n fill: #ff0100;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #ff0100;\n}\n\n.vis-graph-group5 {\n fill: #37d8e6;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #37d8e6;\n}\n\n.vis-graph-group6 {\n fill: #042662;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #042662;\n}\n\n.vis-graph-group7 {\n fill:#00ff26;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #00ff26;\n}\n\n.vis-graph-group8 {\n fill:#ff00ff;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #ff00ff;\n}\n\n.vis-graph-group9 {\n fill: #8f3938;\n fill-opacity:0;\n stroke-width:2px;\n stroke: #8f3938;\n}\n\n.vis-timeline .vis-fill {\n fill-opacity:0.1;\n stroke: none;\n}\n\n\n.vis-timeline .vis-bar {\n fill-opacity:0.5;\n stroke-width:1px;\n}\n\n.vis-timeline .vis-point {\n stroke-width:2px;\n fill-opacity:1.0;\n}\n\n\n.vis-timeline .vis-legend-background {\n stroke-width:1px;\n fill-opacity:0.9;\n fill: #ffffff;\n stroke: #c2c2c2;\n}\n\n\n.vis-timeline .vis-outline {\n stroke-width:1px;\n fill-opacity:1;\n fill: #ffffff;\n stroke: #e5e5e5;\n}\n\n.vis-timeline .vis-icon-fill {\n fill-opacity:0.3;\n stroke: none;\n}\n","\n.vis-panel.vis-background.vis-horizontal .vis-grid.vis-horizontal {\n position: absolute;\n width: 100%;\n height: 0;\n border-bottom: 1px solid;\n}\n\n.vis-panel.vis-background.vis-horizontal .vis-grid.vis-minor {\n border-color: #e5e5e5;\n}\n\n.vis-panel.vis-background.vis-horizontal .vis-grid.vis-major {\n border-color: #bfbfbf;\n}\n\n\n.vis-data-axis .vis-y-axis.vis-major {\n width: 100%;\n position: absolute;\n color: #4d4d4d;\n white-space: nowrap;\n}\n\n.vis-data-axis .vis-y-axis.vis-major.vis-measure {\n padding: 0;\n margin: 0;\n border: 0;\n visibility: hidden;\n width: auto;\n}\n\n\n.vis-data-axis .vis-y-axis.vis-minor {\n position: absolute;\n width: 100%;\n color: #bebebe;\n white-space: nowrap;\n}\n\n.vis-data-axis .vis-y-axis.vis-minor.vis-measure {\n padding: 0;\n margin: 0;\n border: 0;\n visibility: hidden;\n width: auto;\n}\n\n.vis-data-axis .vis-y-axis.vis-title {\n position: absolute;\n color: #4d4d4d;\n white-space: nowrap;\n bottom: 20px;\n text-align: center;\n}\n\n.vis-data-axis .vis-y-axis.vis-title.vis-measure {\n padding: 0;\n margin: 0;\n visibility: hidden;\n width: auto;\n}\n\n.vis-data-axis .vis-y-axis.vis-title.vis-left {\n bottom: 0;\n -webkit-transform-origin: left top;\n -moz-transform-origin: left top;\n -ms-transform-origin: left top;\n -o-transform-origin: left top;\n transform-origin: left bottom;\n -webkit-transform: rotate(-90deg);\n -moz-transform: rotate(-90deg);\n -ms-transform: rotate(-90deg);\n -o-transform: rotate(-90deg);\n transform: rotate(-90deg);\n}\n\n.vis-data-axis .vis-y-axis.vis-title.vis-right {\n bottom: 0;\n -webkit-transform-origin: right bottom;\n -moz-transform-origin: right bottom;\n -ms-transform-origin: right bottom;\n -o-transform-origin: right bottom;\n transform-origin: right bottom;\n -webkit-transform: rotate(90deg);\n -moz-transform: rotate(90deg);\n -ms-transform: rotate(90deg);\n -o-transform: rotate(90deg);\n transform: rotate(90deg);\n}\n\n.vis-legend {\n background-color: rgba(247, 252, 255, 0.65);\n padding: 5px;\n border: 1px solid #b3b3b3;\n box-shadow: 2px 2px 10px rgba(154, 154, 154, 0.55);\n}\n\n.vis-legend-text {\n /*font-size: 10px;*/\n white-space: nowrap;\n display: inline-block\n}","\n.vis-itemset {\n position: relative;\n padding: 0;\n margin: 0;\n\n box-sizing: border-box;\n}\n\n.vis-itemset .vis-background,\n.vis-itemset .vis-foreground {\n position: absolute;\n width: 100%;\n height: 100%;\n overflow: visible;\n}\n\n.vis-axis {\n position: absolute;\n width: 100%;\n height: 0;\n left: 0;\n z-index: 1;\n}\n\n.vis-foreground .vis-group {\n position: relative;\n box-sizing: border-box;\n border-bottom: 1px solid #bfbfbf;\n}\n\n.vis-foreground .vis-group:last-child {\n border-bottom: none;\n}\n\n.vis-nesting-group {\n cursor: pointer;\n}\n\n.vis-label.vis-nested-group.vis-group-level-unknown-but-gte1 {\n background: #f5f5f5;\n}\n.vis-label.vis-nested-group.vis-group-level-0 {\n background-color: #ffffff;\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-0 .vis-inner {\n padding-left: 0;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-0 .vis-inner {\n padding-right: 0;\n}\n.vis-label.vis-nested-group.vis-group-level-1 {\n background-color: rgba(0, 0, 0, 0.05);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-1 .vis-inner {\n padding-left: 15px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-1 .vis-inner {\n padding-right: 15px;\n}\n.vis-label.vis-nested-group.vis-group-level-2 {\n background-color: rgba(0, 0, 0, 0.1);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-2 .vis-inner {\n padding-left: 30px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-2 .vis-inner {\n padding-right: 30px;\n}\n.vis-label.vis-nested-group.vis-group-level-3 {\n background-color: rgba(0, 0, 0, 0.15);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-3 .vis-inner {\n padding-left: 45px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-3 .vis-inner {\n padding-right: 45px;\n}\n.vis-label.vis-nested-group.vis-group-level-4 {\n background-color: rgba(0, 0, 0, 0.2);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-4 .vis-inner {\n padding-left: 60px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-4 .vis-inner {\n padding-right: 60px;\n}\n.vis-label.vis-nested-group.vis-group-level-5 {\n background-color: rgba(0, 0, 0, 0.25);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-5 .vis-inner {\n padding-left: 75px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-5 .vis-inner {\n padding-right: 75px;\n}\n.vis-label.vis-nested-group.vis-group-level-6 {\n background-color: rgba(0, 0, 0, 0.3);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-6 .vis-inner {\n padding-left: 90px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-6 .vis-inner {\n padding-right: 90px;\n}\n.vis-label.vis-nested-group.vis-group-level-7 {\n background-color: rgba(0, 0, 0, 0.35);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-7 .vis-inner {\n padding-left: 105px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-7 .vis-inner {\n padding-right: 105px;\n}\n.vis-label.vis-nested-group.vis-group-level-8 {\n background-color: rgba(0, 0, 0, 0.4);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-8 .vis-inner {\n padding-left: 120px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-8 .vis-inner {\n padding-right: 120px;\n}\n.vis-label.vis-nested-group.vis-group-level-9 {\n background-color: rgba(0, 0, 0, 0.45);\n}\n.vis-ltr .vis-label.vis-nested-group.vis-group-level-9 .vis-inner {\n padding-left: 135px;\n}\n.vis-rtl .vis-label.vis-nested-group.vis-group-level-9 .vis-inner {\n padding-right: 135px;\n}\n/* default takes over beginning with level-10 (thats why we add .vis-nested-group\n to the selectors above, to have higher specifity than these rules for the defaults) */\n.vis-label.vis-nested-group {\n background-color: rgba(0, 0, 0, 0.5);\n}\n.vis-ltr .vis-label.vis-nested-group .vis-inner {\n padding-left: 150px;\n}\n.vis-rtl .vis-label.vis-nested-group .vis-inner {\n padding-right: 150px;\n}\n\n.vis-group-level-unknown-but-gte1 {\n border: 1px solid red;\n}\n\n/* expanded/collapsed indicators */\n.vis-label.vis-nesting-group:before,\n.vis-label.vis-nesting-group:before {\n display: inline-block;\n width: 15px;\n}\n.vis-label.vis-nesting-group.expanded:before {\n content: \"\\25BC\";\n}\n.vis-label.vis-nesting-group.collapsed:before {\n content: \"\\25B6\";\n}\n.vis-rtl .vis-label.vis-nesting-group.collapsed:before {\n content: \"\\25C0\";\n}\n/* compensate missing expanded/collapsed indicator, but only at levels > 0 */\n.vis-ltr .vis-label:not(.vis-nesting-group):not(.vis-group-level-0) {\n padding-left: 15px;\n}\n.vis-rtl .vis-label:not(.vis-nesting-group):not(.vis-group-level-0) {\n padding-right: 15px;\n}\n\n.vis-overlay {\n position: absolute;\n top: 0;\n left: 0;\n width: 100%;\n height: 100%;\n z-index: 10;\n}","\n.vis-labelset {\n position: relative;\n\n overflow: hidden;\n\n box-sizing: border-box;\n}\n\n.vis-labelset .vis-label {\n position: relative;\n left: 0;\n top: 0;\n width: 100%;\n color: #4d4d4d;\n\n box-sizing: border-box;\n}\n\n.vis-labelset .vis-label {\n border-bottom: 1px solid #bfbfbf;\n}\n\n.vis-labelset .vis-label.draggable {\n cursor: pointer;\n}\n\n.vis-group-is-dragging {\n background: rgba(0, 0, 0, .1);\n}\n\n.vis-labelset .vis-label:last-child {\n border-bottom: none;\n}\n\n.vis-labelset .vis-label .vis-inner {\n display: inline-block;\n padding: 5px;\n}\n\n.vis-labelset .vis-label .vis-inner.vis-hidden {\n padding: 0;\n}\n",".vis-time-axis {\n position: relative;\n overflow: hidden;\n}\n\n.vis-time-axis.vis-foreground {\n top: 0;\n left: 0;\n width: 100%;\n}\n\n.vis-time-axis.vis-background {\n position: absolute;\n top: 0;\n left: 0;\n width: 100%;\n height: 100%;\n}\n\n.vis-time-axis .vis-text {\n position: absolute;\n color: #4d4d4d;\n padding: 3px;\n overflow: hidden;\n box-sizing: border-box;\n\n white-space: nowrap;\n}\n\n.vis-time-axis .vis-text.vis-measure {\n position: absolute;\n padding-left: 0;\n padding-right: 0;\n margin-left: 0;\n margin-right: 0;\n visibility: hidden;\n}\n\n.vis-time-axis .vis-grid.vis-vertical {\n position: absolute;\n border-left: 1px solid;\n}\n\n.vis-time-axis .vis-grid.vis-vertical-rtl {\n position: absolute;\n border-right: 1px solid;\n}\n\n.vis-time-axis .vis-grid.vis-minor {\n border-color: #e5e5e5;\n}\n\n.vis-time-axis .vis-grid.vis-major {\n border-color: #bfbfbf;\n}\n","\n.vis-item {\n position: absolute;\n color: #1A1A1A;\n border-color: #97B0F8;\n border-width: 1px;\n background-color: #D5DDF6;\n display: inline-block;\n z-index: 1;\n /*overflow: hidden;*/\n}\n\n.vis-item.vis-selected {\n border-color: #FFC200;\n background-color: #FFF785;\n\n /* z-index must be higher than the z-index of custom time bar and current time bar */\n z-index: 2;\n}\n\n.vis-editable.vis-selected {\n cursor: move;\n}\n\n.vis-item.vis-point.vis-selected {\n background-color: #FFF785;\n}\n\n.vis-item.vis-box {\n text-align: center;\n border-style: solid;\n border-radius: 2px;\n}\n\n.vis-item.vis-point {\n background: none;\n}\n\n.vis-item.vis-dot {\n position: absolute;\n padding: 0;\n border-width: 4px;\n border-style: solid;\n border-radius: 4px;\n}\n\n.vis-item.vis-range {\n border-style: solid;\n border-radius: 2px;\n box-sizing: border-box;\n}\n\n.vis-item.vis-background {\n border: none;\n background-color: rgba(213, 221, 246, 0.4);\n box-sizing: border-box;\n padding: 0;\n margin: 0;\n}\n\n.vis-item .vis-item-overflow {\n position: relative;\n width: 100%;\n height: 100%;\n padding: 0;\n margin: 0;\n overflow: hidden;\n}\n\n.vis-item-visible-frame {\n white-space: nowrap;\n}\n\n.vis-item.vis-range .vis-item-content {\n position: relative;\n display: inline-block;\n}\n\n.vis-item.vis-background .vis-item-content {\n position: absolute;\n display: inline-block;\n}\n\n.vis-item.vis-line {\n padding: 0;\n position: absolute;\n width: 0;\n border-left-width: 1px;\n border-left-style: solid;\n}\n\n.vis-item .vis-item-content {\n white-space: nowrap;\n box-sizing: border-box;\n padding: 5px;\n}\n\n.vis-item .vis-onUpdateTime-tooltip {\n position: absolute;\n background: #4f81bd;\n color: white;\n width: 200px;\n text-align: center;\n white-space: nowrap;\n padding: 5px;\n border-radius: 1px;\n transition: 0.4s;\n -o-transition: 0.4s;\n -moz-transition: 0.4s;\n -webkit-transition: 0.4s;\n}\n\n.vis-item .vis-delete, .vis-item .vis-delete-rtl {\n position: absolute;\n top: 0px;\n width: 24px;\n height: 24px;\n box-sizing: border-box;\n padding: 0px 5px;\n cursor: pointer;\n\n -webkit-transition: background 0.2s linear;\n -moz-transition: background 0.2s linear;\n -ms-transition: background 0.2s linear;\n -o-transition: background 0.2s linear;\n transition: background 0.2s linear;\n}\n\n.vis-item .vis-delete {\n right: -24px;\n}\n\n.vis-item .vis-delete-rtl {\n left: -24px;\n}\n\n.vis-item .vis-delete:after, .vis-item .vis-delete-rtl:after {\n content: \"\\00D7\"; /* MULTIPLICATION SIGN */\n color: red;\n font-family: arial, sans-serif;\n font-size: 22px;\n font-weight: bold;\n\n -webkit-transition: color 0.2s linear;\n -moz-transition: color 0.2s linear;\n -ms-transition: color 0.2s linear;\n -o-transition: color 0.2s linear;\n transition: color 0.2s linear;\n}\n\n.vis-item .vis-delete:hover, .vis-item .vis-delete-rtl:hover {\n background: red;\n}\n\n.vis-item .vis-delete:hover:after, .vis-item .vis-delete-rtl:hover:after {\n color: white;\n}\n\n.vis-item .vis-drag-center {\n position: absolute;\n width: 100%;\n height: 100%;\n top: 0;\n left: 0px;\n cursor: move;\n}\n\n.vis-item.vis-range .vis-drag-left {\n position: absolute;\n width: 24px;\n max-width: 20%;\n min-width: 2px;\n height: 100%;\n top: 0;\n left: -4px;\n\n cursor: w-resize;\n}\n\n.vis-item.vis-range .vis-drag-right {\n position: absolute;\n width: 24px;\n max-width: 20%;\n min-width: 2px;\n height: 100%;\n top: 0;\n right: -4px;\n\n cursor: e-resize;\n}\n\n.vis-range.vis-item.vis-readonly .vis-drag-left,\n.vis-range.vis-item.vis-readonly .vis-drag-right {\n cursor: auto;\n}\n\n.vis-item.vis-cluster {\n vertical-align: center;\n text-align: center;\n border-style: solid;\n border-radius: 2px;\n}\n\n.vis-item.vis-cluster-line {\n padding: 0;\n position: absolute;\n width: 0;\n border-left-width: 1px;\n border-left-style: solid;\n}\n\n.vis-item.vis-cluster-dot {\n position: absolute;\n padding: 0;\n border-width: 4px;\n border-style: solid;\n border-radius: 4px;\n}"]} \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.js b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.js new file mode 100644 index 0000000000000..df730409ee491 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/vis-timeline-graph2d.min.js @@ -0,0 +1,60 @@ +/** + * vis-timeline and vis-graph2d + * https://visjs.github.io/vis-timeline/ + * + * Create a fully customizable, interactive timeline with items and ranges. + * + * @version 7.3.4 + * @date 2020-03-18T17:03:58.105Z + * + * @copyright (c) 2011-2017 Almende B.V, http://almende.com + * @copyright (c) 2017-2019 visjs contributors, https://github.com/visjs + * + * @license + * vis.js is dual licensed under both + * + * 1. The Apache 2.0 License + * http://www.apache.org/licenses/LICENSE-2.0 + * + * and + * + * 2. The MIT License + * http://opensource.org/licenses/MIT + * + * vis.js may be distributed under either license. + */ +!function(t,e){"object"==typeof exports&&"undefined"!=typeof module?e(exports):"function"==typeof define&&define.amd?define(["exports"],e):e((t=t||self).vis=t.vis||{})}(this,(function(t){"use strict";var e="undefined"!=typeof globalThis?globalThis:"undefined"!=typeof window?window:"undefined"!=typeof global?global:"undefined"!=typeof self?self:{};function i(){throw new Error("Dynamic requires are not currently supported by @rollup/plugin-commonjs")}function n(t,e){return t(e={exports:{}},e.exports),e.exports}function o(t){return t&&t.default||t}var s=n((function(t,e){t.exports=function(){var e,n;function o(){return e.apply(null,arguments)}function s(t){return t instanceof Array||"[object Array]"===Object.prototype.toString.call(t)}function r(t){return null!=t&&"[object Object]"===Object.prototype.toString.call(t)}function a(t){return void 0===t}function l(t){return"number"==typeof t||"[object Number]"===Object.prototype.toString.call(t)}function h(t){return t instanceof Date||"[object Date]"===Object.prototype.toString.call(t)}function d(t,e){var i,n=[];for(i=0;i>>0,n=0;n0)for(i=0;i=0?i?"+":"":"-")+Math.pow(10,Math.max(0,o)).toString().substr(1)+n}var T=/(\[[^\[]*\])|(\\)?([Hh]mm(ss)?|Mo|MM?M?M?|Do|DDDo|DD?D?D?|ddd?d?|do?|w[o|w]?|W[o|W]?|Qo?|YYYYYY|YYYYY|YYYY|YY|gg(ggg?)?|GG(GGG?)?|e|E|a|A|hh?|HH?|kk?|mm?|ss?|S{1,9}|x|X|zz?|ZZ?|.)/g,Y=/(\[[^\[]*\])|(\\)?(LTS|LT|LL?L?L?|l{1,4})/g,E={},R={};function U(t,e,i,n){var o=n;"string"==typeof n&&(o=function(){return this[n]()}),t&&(R[t]=o),e&&(R[e[0]]=function(){return X(o.apply(this,arguments),e[1],e[2])}),i&&(R[i]=function(){return this.localeData().ordinal(o.apply(this,arguments),t)})}function z(t,e){return t.isValid()?(e=V(e,t.localeData()),E[e]=E[e]||function(t){var e,i,n,o=t.match(T);for(e=0,i=o.length;e=0&&Y.test(t);)t=t.replace(Y,n),Y.lastIndex=0,i-=1;return t}var j=/\d/,H=/\d\d/,P=/\d{3}/,J=/\d{4}/,K=/[+-]?\d{6}/,q=/\d\d?/,$=/\d\d\d\d?/,tt=/\d\d\d\d\d\d?/,et=/\d{1,3}/,it=/\d{1,4}/,nt=/[+-]?\d{1,6}/,ot=/\d+/,st=/[+-]?\d+/,rt=/Z|[+-]\d\d:?\d\d/gi,at=/Z|[+-]\d\d(?::?\d\d)?/gi,lt=/[0-9]{0,256}['a-z\u00A0-\u05FF\u0700-\uD7FF\uF900-\uFDCF\uFDF0-\uFF07\uFF10-\uFFEF]{1,256}|[\u0600-\u06FF\/]{1,256}(\s*?[\u0600-\u06FF]{1,256}){1,2}/i,ht={};function dt(t,e,i){ht[t]=S(e)?e:function(t,n){return t&&i?i:e}}function ut(t,e){return u(ht,t)?ht[t](e._strict,e._locale):new RegExp(ct(t.replace("\\","").replace(/\\(\[)|\\(\])|\[([^\]\[]*)\]|\\(.)/g,(function(t,e,i,n,o){return e||i||n||o}))))}function ct(t){return t.replace(/[-\/\\^$*+?.()|[\]{}]/g,"\\$&")}var pt={};function mt(t,e){var i,n=e;for("string"==typeof t&&(t=[t]),l(e)&&(n=function(t,i){i[e]=k(t)}),i=0;i68?1900:2e3)};var yt,xt=wt("FullYear",!0);function wt(t,e){return function(i){return null!=i?(kt(this,t,i),o.updateOffset(this,e),this):_t(this,t)}}function _t(t,e){return t.isValid()?t._d["get"+(t._isUTC?"UTC":"")+e]():NaN}function kt(t,e,i){t.isValid()&&!isNaN(i)&&("FullYear"===e&&bt(t.year())&&1===t.month()&&29===t.date()?t._d["set"+(t._isUTC?"UTC":"")+e](i,t.month(),Dt(i,t.month())):t._d["set"+(t._isUTC?"UTC":"")+e](i))}function Dt(t,e){if(isNaN(t)||isNaN(e))return NaN;var i,n=(e%(i=12)+i)%i;return t+=(e-n)/12,1===n?bt(t)?29:28:31-n%7%2}yt=Array.prototype.indexOf?Array.prototype.indexOf:function(t){var e;for(e=0;e=0?(a=new Date(t+400,e,i,n,o,s,r),isFinite(a.getFullYear())&&a.setFullYear(t)):a=new Date(t,e,i,n,o,s,r),a}function Bt(t){var e;if(t<100&&t>=0){var i=Array.prototype.slice.call(arguments);i[0]=t+400,e=new Date(Date.UTC.apply(null,i)),isFinite(e.getUTCFullYear())&&e.setUTCFullYear(t)}else e=new Date(Date.UTC.apply(null,arguments));return e}function Ot(t,e,i){var n=7+e-i;return-(7+Bt(t,0,n).getUTCDay()-e)%7+n-1}function Wt(t,e,i,n,o){var s,r,a=1+7*(e-1)+(7+i-n)%7+Ot(t,n,o);return a<=0?r=vt(s=t-1)+a:a>vt(t)?(s=t+1,r=a-vt(t)):(s=t,r=a),{year:s,dayOfYear:r}}function At(t,e,i){var n,o,s=Ot(t.year(),e,i),r=Math.floor((t.dayOfYear()-s-1)/7)+1;return r<1?n=r+Xt(o=t.year()-1,e,i):r>Xt(t.year(),e,i)?(n=r-Xt(t.year(),e,i),o=t.year()+1):(o=t.year(),n=r),{week:n,year:o}}function Xt(t,e,i){var n=Ot(t,e,i),o=Ot(t+1,e,i);return(vt(t)-n+o)/7}function Tt(t,e){return t.slice(e,7).concat(t.slice(0,e))}U("w",["ww",2],"wo","week"),U("W",["WW",2],"Wo","isoWeek"),Z("week","w"),Z("isoWeek","W"),A("week",5),A("isoWeek",5),dt("w",q),dt("ww",q,H),dt("W",q),dt("WW",q,H),ft(["w","ww","W","WW"],(function(t,e,i,n){e[n.substr(0,1)]=k(t)})),U("d",0,"do","day"),U("dd",0,0,(function(t){return this.localeData().weekdaysMin(this,t)})),U("ddd",0,0,(function(t){return this.localeData().weekdaysShort(this,t)})),U("dddd",0,0,(function(t){return this.localeData().weekdays(this,t)})),U("e",0,0,"weekday"),U("E",0,0,"isoWeekday"),Z("day","d"),Z("weekday","e"),Z("isoWeekday","E"),A("day",11),A("weekday",11),A("isoWeekday",11),dt("d",q),dt("e",q),dt("E",q),dt("dd",(function(t,e){return e.weekdaysMinRegex(t)})),dt("ddd",(function(t,e){return e.weekdaysShortRegex(t)})),dt("dddd",(function(t,e){return e.weekdaysRegex(t)})),ft(["dd","ddd","dddd"],(function(t,e,i,n){var o=i._locale.weekdaysParse(t,n,i._strict);null!=o?e.d=o:m(i).invalidWeekday=t})),ft(["d","e","E"],(function(t,e,i,n){e[n]=k(t)}));var Yt="Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),Et="Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),Rt="Su_Mo_Tu_We_Th_Fr_Sa".split("_");function Ut(t,e,i){var n,o,s,r=t.toLocaleLowerCase();if(!this._weekdaysParse)for(this._weekdaysParse=[],this._shortWeekdaysParse=[],this._minWeekdaysParse=[],n=0;n<7;++n)s=p([2e3,1]).day(n),this._minWeekdaysParse[n]=this.weekdaysMin(s,"").toLocaleLowerCase(),this._shortWeekdaysParse[n]=this.weekdaysShort(s,"").toLocaleLowerCase(),this._weekdaysParse[n]=this.weekdays(s,"").toLocaleLowerCase();return i?"dddd"===e?-1!==(o=yt.call(this._weekdaysParse,r))?o:null:"ddd"===e?-1!==(o=yt.call(this._shortWeekdaysParse,r))?o:null:-1!==(o=yt.call(this._minWeekdaysParse,r))?o:null:"dddd"===e?-1!==(o=yt.call(this._weekdaysParse,r))||-1!==(o=yt.call(this._shortWeekdaysParse,r))||-1!==(o=yt.call(this._minWeekdaysParse,r))?o:null:"ddd"===e?-1!==(o=yt.call(this._shortWeekdaysParse,r))||-1!==(o=yt.call(this._weekdaysParse,r))||-1!==(o=yt.call(this._minWeekdaysParse,r))?o:null:-1!==(o=yt.call(this._minWeekdaysParse,r))||-1!==(o=yt.call(this._weekdaysParse,r))||-1!==(o=yt.call(this._shortWeekdaysParse,r))?o:null}var zt=lt,Vt=lt,jt=lt;function Ht(){function t(t,e){return e.length-t.length}var e,i,n,o,s,r=[],a=[],l=[],h=[];for(e=0;e<7;e++)i=p([2e3,1]).day(e),n=this.weekdaysMin(i,""),o=this.weekdaysShort(i,""),s=this.weekdays(i,""),r.push(n),a.push(o),l.push(s),h.push(n),h.push(o),h.push(s);for(r.sort(t),a.sort(t),l.sort(t),h.sort(t),e=0;e<7;e++)a[e]=ct(a[e]),l[e]=ct(l[e]),h[e]=ct(h[e]);this._weekdaysRegex=new RegExp("^("+h.join("|")+")","i"),this._weekdaysShortRegex=this._weekdaysRegex,this._weekdaysMinRegex=this._weekdaysRegex,this._weekdaysStrictRegex=new RegExp("^("+l.join("|")+")","i"),this._weekdaysShortStrictRegex=new RegExp("^("+a.join("|")+")","i"),this._weekdaysMinStrictRegex=new RegExp("^("+r.join("|")+")","i")}function Pt(){return this.hours()%12||12}function Jt(t,e){U(t,0,0,(function(){return this.localeData().meridiem(this.hours(),this.minutes(),e)}))}function Kt(t,e){return e._meridiemParse}U("H",["HH",2],0,"hour"),U("h",["hh",2],0,Pt),U("k",["kk",2],0,(function(){return this.hours()||24})),U("hmm",0,0,(function(){return""+Pt.apply(this)+X(this.minutes(),2)})),U("hmmss",0,0,(function(){return""+Pt.apply(this)+X(this.minutes(),2)+X(this.seconds(),2)})),U("Hmm",0,0,(function(){return""+this.hours()+X(this.minutes(),2)})),U("Hmmss",0,0,(function(){return""+this.hours()+X(this.minutes(),2)+X(this.seconds(),2)})),Jt("a",!0),Jt("A",!1),Z("hour","h"),A("hour",13),dt("a",Kt),dt("A",Kt),dt("H",q),dt("h",q),dt("k",q),dt("HH",q,H),dt("hh",q,H),dt("kk",q,H),dt("hmm",$),dt("hmmss",tt),dt("Hmm",$),dt("Hmmss",tt),mt(["H","HH"],3),mt(["k","kk"],(function(t,e,i){var n=k(t);e[3]=24===n?0:n})),mt(["a","A"],(function(t,e,i){i._isPm=i._locale.isPM(t),i._meridiem=t})),mt(["h","hh"],(function(t,e,i){e[3]=k(t),m(i).bigHour=!0})),mt("hmm",(function(t,e,i){var n=t.length-2;e[3]=k(t.substr(0,n)),e[4]=k(t.substr(n)),m(i).bigHour=!0})),mt("hmmss",(function(t,e,i){var n=t.length-4,o=t.length-2;e[3]=k(t.substr(0,n)),e[4]=k(t.substr(n,2)),e[5]=k(t.substr(o)),m(i).bigHour=!0})),mt("Hmm",(function(t,e,i){var n=t.length-2;e[3]=k(t.substr(0,n)),e[4]=k(t.substr(n))})),mt("Hmmss",(function(t,e,i){var n=t.length-4,o=t.length-2;e[3]=k(t.substr(0,n)),e[4]=k(t.substr(n,2)),e[5]=k(t.substr(o))}));var qt,$t=wt("Hours",!0),te={calendar:{sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},longDateFormat:{LTS:"h:mm:ss A",LT:"h:mm A",L:"MM/DD/YYYY",LL:"MMMM D, YYYY",LLL:"MMMM D, YYYY h:mm A",LLLL:"dddd, MMMM D, YYYY h:mm A"},invalidDate:"Invalid date",ordinal:"%d",dayOfMonthOrdinalParse:/\d{1,2}/,relativeTime:{future:"in %s",past:"%s ago",s:"a few seconds",ss:"%d seconds",m:"a minute",mm:"%d minutes",h:"an hour",hh:"%d hours",d:"a day",dd:"%d days",M:"a month",MM:"%d months",y:"a year",yy:"%d years"},months:Ct,monthsShort:Lt,week:{dow:0,doy:6},weekdays:Yt,weekdaysMin:Rt,weekdaysShort:Et,meridiemParse:/[ap]\.?m?\.?/i},ee={},ie={};function ne(t){return t?t.toLowerCase().replace("_","-"):t}function oe(e){var n=null;if(!ee[e]&&t&&t.exports)try{n=qt._abbr,i(),se(n)}catch(t){}return ee[e]}function se(t,e){var i;return t&&((i=a(e)?ae(t):re(t,e))?qt=i:"undefined"!=typeof console&&console.warn&&console.warn("Locale "+t+" not found. Did you forget to load it?")),qt._abbr}function re(t,e){if(null!==e){var i,n=te;if(e.abbr=t,null!=ee[t])M("defineLocaleOverride","use moment.updateLocale(localeName, config) to change an existing locale. moment.defineLocale(localeName, config) should only be used for creating a new locale See http://momentjs.com/guides/#/warnings/define-locale/ for more info."),n=ee[t]._config;else if(null!=e.parentLocale)if(null!=ee[e.parentLocale])n=ee[e.parentLocale]._config;else{if(null==(i=oe(e.parentLocale)))return ie[e.parentLocale]||(ie[e.parentLocale]=[]),ie[e.parentLocale].push({name:t,config:e}),null;n=i._config}return ee[t]=new N(Q(n,e)),ie[t]&&ie[t].forEach((function(t){re(t.name,t.config)})),se(t),ee[t]}return delete ee[t],null}function ae(t){var e;if(t&&t._locale&&t._locale._abbr&&(t=t._locale._abbr),!t)return qt;if(!s(t)){if(e=oe(t))return e;t=[t]}return function(t){for(var e,i,n,o,s=0;s0;){if(n=oe(o.slice(0,e).join("-")))return n;if(i&&i.length>=e&&D(o,i,!0)>=e-1)break;e--}s++}return qt}(t)}function le(t){var e,i=t._a;return i&&-2===m(t).overflow&&(e=i[1]<0||i[1]>11?1:i[2]<1||i[2]>Dt(i[0],i[1])?2:i[3]<0||i[3]>24||24===i[3]&&(0!==i[4]||0!==i[5]||0!==i[6])?3:i[4]<0||i[4]>59?4:i[5]<0||i[5]>59?5:i[6]<0||i[6]>999?6:-1,m(t)._overflowDayOfYear&&(e<0||e>2)&&(e=2),m(t)._overflowWeeks&&-1===e&&(e=7),m(t)._overflowWeekday&&-1===e&&(e=8),m(t).overflow=e),t}function he(t,e,i){return null!=t?t:null!=e?e:i}function de(t){var e,i,n,s,r,a=[];if(!t._d){for(n=function(t){var e=new Date(o.now());return t._useUTC?[e.getUTCFullYear(),e.getUTCMonth(),e.getUTCDate()]:[e.getFullYear(),e.getMonth(),e.getDate()]}(t),t._w&&null==t._a[2]&&null==t._a[1]&&function(t){var e,i,n,o,s,r,a,l;if(null!=(e=t._w).GG||null!=e.W||null!=e.E)s=1,r=4,i=he(e.GG,t._a[0],At(Ie(),1,4).year),n=he(e.W,1),((o=he(e.E,1))<1||o>7)&&(l=!0);else{s=t._locale._week.dow,r=t._locale._week.doy;var h=At(Ie(),s,r);i=he(e.gg,t._a[0],h.year),n=he(e.w,h.week),null!=e.d?((o=e.d)<0||o>6)&&(l=!0):null!=e.e?(o=e.e+s,(e.e<0||e.e>6)&&(l=!0)):o=s}n<1||n>Xt(i,s,r)?m(t)._overflowWeeks=!0:null!=l?m(t)._overflowWeekday=!0:(a=Wt(i,n,o,s,r),t._a[0]=a.year,t._dayOfYear=a.dayOfYear)}(t),null!=t._dayOfYear&&(r=he(t._a[0],n[0]),(t._dayOfYear>vt(r)||0===t._dayOfYear)&&(m(t)._overflowDayOfYear=!0),i=Bt(r,0,t._dayOfYear),t._a[1]=i.getUTCMonth(),t._a[2]=i.getUTCDate()),e=0;e<3&&null==t._a[e];++e)t._a[e]=a[e]=n[e];for(;e<7;e++)t._a[e]=a[e]=null==t._a[e]?2===e?1:0:t._a[e];24===t._a[3]&&0===t._a[4]&&0===t._a[5]&&0===t._a[6]&&(t._nextDay=!0,t._a[3]=0),t._d=(t._useUTC?Bt:Zt).apply(null,a),s=t._useUTC?t._d.getUTCDay():t._d.getDay(),null!=t._tzm&&t._d.setUTCMinutes(t._d.getUTCMinutes()-t._tzm),t._nextDay&&(t._a[3]=24),t._w&&void 0!==t._w.d&&t._w.d!==s&&(m(t).weekdayMismatch=!0)}}var ue=/^\s*((?:[+-]\d{6}|\d{4})-(?:\d\d-\d\d|W\d\d-\d|W\d\d|\d\d\d|\d\d))(?:(T| )(\d\d(?::\d\d(?::\d\d(?:[.,]\d+)?)?)?)([\+\-]\d\d(?::?\d\d)?|\s*Z)?)?$/,ce=/^\s*((?:[+-]\d{6}|\d{4})(?:\d\d\d\d|W\d\d\d|W\d\d|\d\d\d|\d\d))(?:(T| )(\d\d(?:\d\d(?:\d\d(?:[.,]\d+)?)?)?)([\+\-]\d\d(?::?\d\d)?|\s*Z)?)?$/,pe=/Z|[+-]\d\d(?::?\d\d)?/,me=[["YYYYYY-MM-DD",/[+-]\d{6}-\d\d-\d\d/],["YYYY-MM-DD",/\d{4}-\d\d-\d\d/],["GGGG-[W]WW-E",/\d{4}-W\d\d-\d/],["GGGG-[W]WW",/\d{4}-W\d\d/,!1],["YYYY-DDD",/\d{4}-\d{3}/],["YYYY-MM",/\d{4}-\d\d/,!1],["YYYYYYMMDD",/[+-]\d{10}/],["YYYYMMDD",/\d{8}/],["GGGG[W]WWE",/\d{4}W\d{3}/],["GGGG[W]WW",/\d{4}W\d{2}/,!1],["YYYYDDD",/\d{7}/]],fe=[["HH:mm:ss.SSSS",/\d\d:\d\d:\d\d\.\d+/],["HH:mm:ss,SSSS",/\d\d:\d\d:\d\d,\d+/],["HH:mm:ss",/\d\d:\d\d:\d\d/],["HH:mm",/\d\d:\d\d/],["HHmmss.SSSS",/\d\d\d\d\d\d\.\d+/],["HHmmss,SSSS",/\d\d\d\d\d\d,\d+/],["HHmmss",/\d\d\d\d\d\d/],["HHmm",/\d\d\d\d/],["HH",/\d\d/]],ge=/^\/?Date\((\-?\d+)/i;function ve(t){var e,i,n,o,s,r,a=t._i,l=ue.exec(a)||ce.exec(a);if(l){for(m(t).iso=!0,e=0,i=me.length;e0&&m(t).unusedInput.push(r),a=a.slice(a.indexOf(i)+i.length),h+=i.length),R[s]?(i?m(t).empty=!1:m(t).unusedTokens.push(s),gt(s,i,t)):t._strict&&!i&&m(t).unusedTokens.push(s);m(t).charsLeftOver=l-h,a.length>0&&m(t).unusedInput.push(a),t._a[3]<=12&&!0===m(t).bigHour&&t._a[3]>0&&(m(t).bigHour=void 0),m(t).parsedDateParts=t._a.slice(0),m(t).meridiem=t._meridiem,t._a[3]=function(t,e,i){var n;return null==i?e:null!=t.meridiemHour?t.meridiemHour(e,i):null!=t.isPM?((n=t.isPM(i))&&e<12&&(e+=12),n||12!==e||(e=0),e):e}(t._locale,t._a[3],t._meridiem),de(t),le(t)}else we(t);else ve(t)}function ke(t){var e=t._i,i=t._f;return t._locale=t._locale||ae(t._l),null===e||void 0===i&&""===e?g({nullInput:!0}):("string"==typeof e&&(t._i=e=t._locale.preparse(e)),w(e)?new x(le(e)):(h(e)?t._d=e:s(i)?function(t){var e,i,n,o,s;if(0===t._f.length)return m(t).invalidFormat=!0,void(t._d=new Date(NaN));for(o=0;othis?this:t:g()}));function Ge(t,e){var i,n;if(1===e.length&&s(e[0])&&(e=e[0]),!e.length)return Ie();for(i=e[0],n=1;n=0?new Date(t+400,e,i)-126227808e5:new Date(t,e,i).valueOf()}function ti(t,e,i){return t<100&&t>=0?Date.UTC(t+400,e,i)-126227808e5:Date.UTC(t,e,i)}function ei(t,e){U(0,[t,t.length],0,e)}function ii(t,e,i,n,o){var s;return null==t?At(this,n,o).year:(e>(s=Xt(t,n,o))&&(e=s),ni.call(this,t,e,i,n,o))}function ni(t,e,i,n,o){var s=Wt(t,e,i,n,o),r=Bt(s.year,0,s.dayOfYear);return this.year(r.getUTCFullYear()),this.month(r.getUTCMonth()),this.date(r.getUTCDate()),this}U(0,["gg",2],0,(function(){return this.weekYear()%100})),U(0,["GG",2],0,(function(){return this.isoWeekYear()%100})),ei("gggg","weekYear"),ei("ggggg","weekYear"),ei("GGGG","isoWeekYear"),ei("GGGGG","isoWeekYear"),Z("weekYear","gg"),Z("isoWeekYear","GG"),A("weekYear",1),A("isoWeekYear",1),dt("G",st),dt("g",st),dt("GG",q,H),dt("gg",q,H),dt("GGGG",it,J),dt("gggg",it,J),dt("GGGGG",nt,K),dt("ggggg",nt,K),ft(["gggg","ggggg","GGGG","GGGGG"],(function(t,e,i,n){e[n.substr(0,2)]=k(t)})),ft(["gg","GG"],(function(t,e,i,n){e[n]=o.parseTwoDigitYear(t)})),U("Q",0,"Qo","quarter"),Z("quarter","Q"),A("quarter",7),dt("Q",j),mt("Q",(function(t,e){e[1]=3*(k(t)-1)})),U("D",["DD",2],"Do","date"),Z("date","D"),A("date",9),dt("D",q),dt("DD",q,H),dt("Do",(function(t,e){return t?e._dayOfMonthOrdinalParse||e._ordinalParse:e._dayOfMonthOrdinalParseLenient})),mt(["D","DD"],2),mt("Do",(function(t,e){e[2]=k(t.match(q)[0])}));var oi=wt("Date",!0);U("DDD",["DDDD",3],"DDDo","dayOfYear"),Z("dayOfYear","DDD"),A("dayOfYear",4),dt("DDD",et),dt("DDDD",P),mt(["DDD","DDDD"],(function(t,e,i){i._dayOfYear=k(t)})),U("m",["mm",2],0,"minute"),Z("minute","m"),A("minute",14),dt("m",q),dt("mm",q,H),mt(["m","mm"],4);var si=wt("Minutes",!1);U("s",["ss",2],0,"second"),Z("second","s"),A("second",15),dt("s",q),dt("ss",q,H),mt(["s","ss"],5);var ri,ai=wt("Seconds",!1);for(U("S",0,0,(function(){return~~(this.millisecond()/100)})),U(0,["SS",2],0,(function(){return~~(this.millisecond()/10)})),U(0,["SSS",3],0,"millisecond"),U(0,["SSSS",4],0,(function(){return 10*this.millisecond()})),U(0,["SSSSS",5],0,(function(){return 100*this.millisecond()})),U(0,["SSSSSS",6],0,(function(){return 1e3*this.millisecond()})),U(0,["SSSSSSS",7],0,(function(){return 1e4*this.millisecond()})),U(0,["SSSSSSSS",8],0,(function(){return 1e5*this.millisecond()})),U(0,["SSSSSSSSS",9],0,(function(){return 1e6*this.millisecond()})),Z("millisecond","ms"),A("millisecond",16),dt("S",et,j),dt("SS",et,H),dt("SSS",et,P),ri="SSSS";ri.length<=9;ri+="S")dt(ri,ot);function li(t,e){e[6]=k(1e3*("0."+t))}for(ri="S";ri.length<=9;ri+="S")mt(ri,li);var hi=wt("Milliseconds",!1);U("z",0,0,"zoneAbbr"),U("zz",0,0,"zoneName");var di=x.prototype;function ui(t){return t}di.add=Ve,di.calendar=function(t,e){var i=t||Ie(),n=Oe(i,this).startOf("day"),s=o.calendarFormat(this,n)||"sameElse",r=e&&(S(e[s])?e[s].call(this,i):e[s]);return this.format(r||this.localeData().calendar(s,this,Ie(i)))},di.clone=function(){return new x(this)},di.diff=function(t,e,i){var n,o,s;if(!this.isValid())return NaN;if(!(n=Oe(t,this)).isValid())return NaN;switch(o=6e4*(n.utcOffset()-this.utcOffset()),e=B(e)){case"year":s=He(this,n)/12;break;case"month":s=He(this,n);break;case"quarter":s=He(this,n)/3;break;case"second":s=(this-n)/1e3;break;case"minute":s=(this-n)/6e4;break;case"hour":s=(this-n)/36e5;break;case"day":s=(this-n-o)/864e5;break;case"week":s=(this-n-o)/6048e5;break;default:s=this-n}return i?s:_(s)},di.endOf=function(t){var e;if(void 0===(t=B(t))||"millisecond"===t||!this.isValid())return this;var i=this._isUTC?ti:$e;switch(t){case"year":e=i(this.year()+1,0,1)-1;break;case"quarter":e=i(this.year(),this.month()-this.month()%3+3,1)-1;break;case"month":e=i(this.year(),this.month()+1,1)-1;break;case"week":e=i(this.year(),this.month(),this.date()-this.weekday()+7)-1;break;case"isoWeek":e=i(this.year(),this.month(),this.date()-(this.isoWeekday()-1)+7)-1;break;case"day":case"date":e=i(this.year(),this.month(),this.date()+1)-1;break;case"hour":e=this._d.valueOf(),e+=36e5-qe(e+(this._isUTC?0:6e4*this.utcOffset()),36e5)-1;break;case"minute":e=this._d.valueOf(),e+=6e4-qe(e,6e4)-1;break;case"second":e=this._d.valueOf(),e+=1e3-qe(e,1e3)-1}return this._d.setTime(e),o.updateOffset(this,!0),this},di.format=function(t){t||(t=this.isUtc()?o.defaultFormatUtc:o.defaultFormat);var e=z(this,t);return this.localeData().postformat(e)},di.from=function(t,e){return this.isValid()&&(w(t)&&t.isValid()||Ie(t).isValid())?Ye({to:this,from:t}).locale(this.locale()).humanize(!e):this.localeData().invalidDate()},di.fromNow=function(t){return this.from(Ie(),t)},di.to=function(t,e){return this.isValid()&&(w(t)&&t.isValid()||Ie(t).isValid())?Ye({from:this,to:t}).locale(this.locale()).humanize(!e):this.localeData().invalidDate()},di.toNow=function(t){return this.to(Ie(),t)},di.get=function(t){return S(this[t=B(t)])?this[t]():this},di.invalidAt=function(){return m(this).overflow},di.isAfter=function(t,e){var i=w(t)?t:Ie(t);return!(!this.isValid()||!i.isValid())&&("millisecond"===(e=B(e)||"millisecond")?this.valueOf()>i.valueOf():i.valueOf()9999?z(i,e?"YYYYYY-MM-DD[T]HH:mm:ss.SSS[Z]":"YYYYYY-MM-DD[T]HH:mm:ss.SSSZ"):S(Date.prototype.toISOString)?e?this.toDate().toISOString():new Date(this.valueOf()+60*this.utcOffset()*1e3).toISOString().replace("Z",z(i,"Z")):z(i,e?"YYYY-MM-DD[T]HH:mm:ss.SSS[Z]":"YYYY-MM-DD[T]HH:mm:ss.SSSZ")},di.inspect=function(){if(!this.isValid())return"moment.invalid(/* "+this._i+" */)";var t="moment",e="";this.isLocal()||(t=0===this.utcOffset()?"moment.utc":"moment.parseZone",e="Z");var i="["+t+'("]',n=0<=this.year()&&this.year()<=9999?"YYYY":"YYYYYY",o=e+'[")]';return this.format(i+n+"-MM-DD[T]HH:mm:ss.SSS"+o)},di.toJSON=function(){return this.isValid()?this.toISOString():null},di.toString=function(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")},di.unix=function(){return Math.floor(this.valueOf()/1e3)},di.valueOf=function(){return this._d.valueOf()-6e4*(this._offset||0)},di.creationData=function(){return{input:this._i,format:this._f,locale:this._locale,isUTC:this._isUTC,strict:this._strict}},di.year=xt,di.isLeapYear=function(){return bt(this.year())},di.weekYear=function(t){return ii.call(this,t,this.week(),this.weekday(),this.localeData()._week.dow,this.localeData()._week.doy)},di.isoWeekYear=function(t){return ii.call(this,t,this.isoWeek(),this.isoWeekday(),1,4)},di.quarter=di.quarters=function(t){return null==t?Math.ceil((this.month()+1)/3):this.month(3*(t-1)+this.month()%3)},di.month=St,di.daysInMonth=function(){return Dt(this.year(),this.month())},di.week=di.weeks=function(t){var e=this.localeData().week(this);return null==t?e:this.add(7*(t-e),"d")},di.isoWeek=di.isoWeeks=function(t){var e=At(this,1,4).week;return null==t?e:this.add(7*(t-e),"d")},di.weeksInYear=function(){var t=this.localeData()._week;return Xt(this.year(),t.dow,t.doy)},di.isoWeeksInYear=function(){return Xt(this.year(),1,4)},di.date=oi,di.day=di.days=function(t){if(!this.isValid())return null!=t?this:NaN;var e=this._isUTC?this._d.getUTCDay():this._d.getDay();return null!=t?(t=function(t,e){return"string"!=typeof t?t:isNaN(t)?"number"==typeof(t=e.weekdaysParse(t))?t:null:parseInt(t,10)}(t,this.localeData()),this.add(t-e,"d")):e},di.weekday=function(t){if(!this.isValid())return null!=t?this:NaN;var e=(this.day()+7-this.localeData()._week.dow)%7;return null==t?e:this.add(t-e,"d")},di.isoWeekday=function(t){if(!this.isValid())return null!=t?this:NaN;if(null!=t){var e=function(t,e){return"string"==typeof t?e.weekdaysParse(t)%7||7:isNaN(t)?null:t}(t,this.localeData());return this.day(this.day()%7?e:e-7)}return this.day()||7},di.dayOfYear=function(t){var e=Math.round((this.clone().startOf("day")-this.clone().startOf("year"))/864e5)+1;return null==t?e:this.add(t-e,"d")},di.hour=di.hours=$t,di.minute=di.minutes=si,di.second=di.seconds=ai,di.millisecond=di.milliseconds=hi,di.utcOffset=function(t,e,i){var n,s=this._offset||0;if(!this.isValid())return null!=t?this:NaN;if(null!=t){if("string"==typeof t){if(null===(t=Be(at,t)))return this}else Math.abs(t)<16&&!i&&(t*=60);return!this._isUTC&&e&&(n=We(this)),this._offset=t,this._isUTC=!0,null!=n&&this.add(n,"m"),s!==t&&(!e||this._changeInProgress?ze(this,Ye(t-s,"m"),1,!1):this._changeInProgress||(this._changeInProgress=!0,o.updateOffset(this,!0),this._changeInProgress=null)),this}return this._isUTC?s:We(this)},di.utc=function(t){return this.utcOffset(0,t)},di.local=function(t){return this._isUTC&&(this.utcOffset(0,t),this._isUTC=!1,t&&this.subtract(We(this),"m")),this},di.parseZone=function(){if(null!=this._tzm)this.utcOffset(this._tzm,!1,!0);else if("string"==typeof this._i){var t=Be(rt,this._i);null!=t?this.utcOffset(t):this.utcOffset(0,!0)}return this},di.hasAlignedHourOffset=function(t){return!!this.isValid()&&(t=t?Ie(t).utcOffset():0,(this.utcOffset()-t)%60==0)},di.isDST=function(){return this.utcOffset()>this.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()},di.isLocal=function(){return!!this.isValid()&&!this._isUTC},di.isUtcOffset=function(){return!!this.isValid()&&this._isUTC},di.isUtc=Ae,di.isUTC=Ae,di.zoneAbbr=function(){return this._isUTC?"UTC":""},di.zoneName=function(){return this._isUTC?"Coordinated Universal Time":""},di.dates=C("dates accessor is deprecated. Use date instead.",oi),di.months=C("months accessor is deprecated. Use month instead",St),di.years=C("years accessor is deprecated. Use year instead",xt),di.zone=C("moment().zone is deprecated, use moment().utcOffset instead. http://momentjs.com/guides/#/warnings/zone/",(function(t,e){return null!=t?("string"!=typeof t&&(t=-t),this.utcOffset(t,e),this):-this.utcOffset()})),di.isDSTShifted=C("isDSTShifted is deprecated. See http://momentjs.com/guides/#/warnings/dst-shifted/ for more information",(function(){if(!a(this._isDSTShifted))return this._isDSTShifted;var t={};if(b(t,this),(t=ke(t))._a){var e=t._isUTC?p(t._a):Ie(t._a);this._isDSTShifted=this.isValid()&&D(t._a,e.toArray())>0}else this._isDSTShifted=!1;return this._isDSTShifted}));var ci=N.prototype;function pi(t,e,i,n){var o=ae(),s=p().set(n,e);return o[i](s,t)}function mi(t,e,i){if(l(t)&&(e=t,t=void 0),t=t||"",null!=e)return pi(t,e,i,"month");var n,o=[];for(n=0;n<12;n++)o[n]=pi(t,n,i,"month");return o}function fi(t,e,i,n){"boolean"==typeof t?(l(e)&&(i=e,e=void 0),e=e||""):(i=e=t,t=!1,l(e)&&(i=e,e=void 0),e=e||"");var o,s=ae(),r=t?s._week.dow:0;if(null!=i)return pi(e,(i+r)%7,n,"day");var a=[];for(o=0;o<7;o++)a[o]=pi(e,(o+r)%7,n,"day");return a}ci.calendar=function(t,e,i){var n=this._calendar[t]||this._calendar.sameElse;return S(n)?n.call(e,i):n},ci.longDateFormat=function(t){var e=this._longDateFormat[t],i=this._longDateFormat[t.toUpperCase()];return e||!i?e:(this._longDateFormat[t]=i.replace(/MMMM|MM|DD|dddd/g,(function(t){return t.slice(1)})),this._longDateFormat[t])},ci.invalidDate=function(){return this._invalidDate},ci.ordinal=function(t){return this._ordinal.replace("%d",t)},ci.preparse=ui,ci.postformat=ui,ci.relativeTime=function(t,e,i,n){var o=this._relativeTime[i];return S(o)?o(t,e,i,n):o.replace(/%d/i,t)},ci.pastFuture=function(t,e){var i=this._relativeTime[t>0?"future":"past"];return S(i)?i(e):i.replace(/%s/i,e)},ci.set=function(t){var e,i;for(i in t)S(e=t[i])?this[i]=e:this["_"+i]=e;this._config=t,this._dayOfMonthOrdinalParseLenient=new RegExp((this._dayOfMonthOrdinalParse.source||this._ordinalParse.source)+"|"+/\d{1,2}/.source)},ci.months=function(t,e){return t?s(this._months)?this._months[t.month()]:this._months[(this._months.isFormat||It).test(e)?"format":"standalone"][t.month()]:s(this._months)?this._months:this._months.standalone},ci.monthsShort=function(t,e){return t?s(this._monthsShort)?this._monthsShort[t.month()]:this._monthsShort[It.test(e)?"format":"standalone"][t.month()]:s(this._monthsShort)?this._monthsShort:this._monthsShort.standalone},ci.monthsParse=function(t,e,i){var n,o,s;if(this._monthsParseExact)return Gt.call(this,t,e,i);for(this._monthsParse||(this._monthsParse=[],this._longMonthsParse=[],this._shortMonthsParse=[]),n=0;n<12;n++){if(o=p([2e3,n]),i&&!this._longMonthsParse[n]&&(this._longMonthsParse[n]=new RegExp("^"+this.months(o,"").replace(".","")+"$","i"),this._shortMonthsParse[n]=new RegExp("^"+this.monthsShort(o,"").replace(".","")+"$","i")),i||this._monthsParse[n]||(s="^"+this.months(o,"")+"|^"+this.monthsShort(o,""),this._monthsParse[n]=new RegExp(s.replace(".",""),"i")),i&&"MMMM"===e&&this._longMonthsParse[n].test(t))return n;if(i&&"MMM"===e&&this._shortMonthsParse[n].test(t))return n;if(!i&&this._monthsParse[n].test(t))return n}},ci.monthsRegex=function(t){return this._monthsParseExact?(u(this,"_monthsRegex")||Ft.call(this),t?this._monthsStrictRegex:this._monthsRegex):(u(this,"_monthsRegex")||(this._monthsRegex=Nt),this._monthsStrictRegex&&t?this._monthsStrictRegex:this._monthsRegex)},ci.monthsShortRegex=function(t){return this._monthsParseExact?(u(this,"_monthsRegex")||Ft.call(this),t?this._monthsShortStrictRegex:this._monthsShortRegex):(u(this,"_monthsShortRegex")||(this._monthsShortRegex=Qt),this._monthsShortStrictRegex&&t?this._monthsShortStrictRegex:this._monthsShortRegex)},ci.week=function(t){return At(t,this._week.dow,this._week.doy).week},ci.firstDayOfYear=function(){return this._week.doy},ci.firstDayOfWeek=function(){return this._week.dow},ci.weekdays=function(t,e){var i=s(this._weekdays)?this._weekdays:this._weekdays[t&&!0!==t&&this._weekdays.isFormat.test(e)?"format":"standalone"];return!0===t?Tt(i,this._week.dow):t?i[t.day()]:i},ci.weekdaysMin=function(t){return!0===t?Tt(this._weekdaysMin,this._week.dow):t?this._weekdaysMin[t.day()]:this._weekdaysMin},ci.weekdaysShort=function(t){return!0===t?Tt(this._weekdaysShort,this._week.dow):t?this._weekdaysShort[t.day()]:this._weekdaysShort},ci.weekdaysParse=function(t,e,i){var n,o,s;if(this._weekdaysParseExact)return Ut.call(this,t,e,i);for(this._weekdaysParse||(this._weekdaysParse=[],this._minWeekdaysParse=[],this._shortWeekdaysParse=[],this._fullWeekdaysParse=[]),n=0;n<7;n++){if(o=p([2e3,1]).day(n),i&&!this._fullWeekdaysParse[n]&&(this._fullWeekdaysParse[n]=new RegExp("^"+this.weekdays(o,"").replace(".","\\.?")+"$","i"),this._shortWeekdaysParse[n]=new RegExp("^"+this.weekdaysShort(o,"").replace(".","\\.?")+"$","i"),this._minWeekdaysParse[n]=new RegExp("^"+this.weekdaysMin(o,"").replace(".","\\.?")+"$","i")),this._weekdaysParse[n]||(s="^"+this.weekdays(o,"")+"|^"+this.weekdaysShort(o,"")+"|^"+this.weekdaysMin(o,""),this._weekdaysParse[n]=new RegExp(s.replace(".",""),"i")),i&&"dddd"===e&&this._fullWeekdaysParse[n].test(t))return n;if(i&&"ddd"===e&&this._shortWeekdaysParse[n].test(t))return n;if(i&&"dd"===e&&this._minWeekdaysParse[n].test(t))return n;if(!i&&this._weekdaysParse[n].test(t))return n}},ci.weekdaysRegex=function(t){return this._weekdaysParseExact?(u(this,"_weekdaysRegex")||Ht.call(this),t?this._weekdaysStrictRegex:this._weekdaysRegex):(u(this,"_weekdaysRegex")||(this._weekdaysRegex=zt),this._weekdaysStrictRegex&&t?this._weekdaysStrictRegex:this._weekdaysRegex)},ci.weekdaysShortRegex=function(t){return this._weekdaysParseExact?(u(this,"_weekdaysRegex")||Ht.call(this),t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex):(u(this,"_weekdaysShortRegex")||(this._weekdaysShortRegex=Vt),this._weekdaysShortStrictRegex&&t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex)},ci.weekdaysMinRegex=function(t){return this._weekdaysParseExact?(u(this,"_weekdaysRegex")||Ht.call(this),t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex):(u(this,"_weekdaysMinRegex")||(this._weekdaysMinRegex=jt),this._weekdaysMinStrictRegex&&t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex)},ci.isPM=function(t){return"p"===(t+"").toLowerCase().charAt(0)},ci.meridiem=function(t,e,i){return t>11?i?"pm":"PM":i?"am":"AM"},se("en",{dayOfMonthOrdinalParse:/\d{1,2}(th|st|nd|rd)/,ordinal:function(t){var e=t%10;return t+(1===k(t%100/10)?"th":1===e?"st":2===e?"nd":3===e?"rd":"th")}}),o.lang=C("moment.lang is deprecated. Use moment.locale instead.",se),o.langData=C("moment.langData is deprecated. Use moment.localeData instead.",ae);var gi=Math.abs;function vi(t,e,i,n){var o=Ye(e,i);return t._milliseconds+=n*o._milliseconds,t._days+=n*o._days,t._months+=n*o._months,t._bubble()}function bi(t){return t<0?Math.floor(t):Math.ceil(t)}function yi(t){return 4800*t/146097}function xi(t){return 146097*t/4800}function wi(t){return function(){return this.as(t)}}var _i=wi("ms"),ki=wi("s"),Di=wi("m"),Ii=wi("h"),Ci=wi("d"),Li=wi("w"),Gi=wi("M"),Mi=wi("Q"),Si=wi("y");function Qi(t){return function(){return this.isValid()?this._data[t]:NaN}}var Ni=Qi("milliseconds"),Fi=Qi("seconds"),Zi=Qi("minutes"),Bi=Qi("hours"),Oi=Qi("days"),Wi=Qi("months"),Ai=Qi("years"),Xi=Math.round,Ti={ss:44,s:45,m:45,h:22,d:26,M:11};function Yi(t,e,i,n,o){return o.relativeTime(e||1,!!i,t,n)}function Ei(t,e,i){var n=Ye(t).abs(),o=Xi(n.as("s")),s=Xi(n.as("m")),r=Xi(n.as("h")),a=Xi(n.as("d")),l=Xi(n.as("M")),h=Xi(n.as("y")),d=o<=Ti.ss&&["s",o]||o0,d[4]=i,Yi.apply(null,d)}var Ri=Math.abs;function Ui(t){return(t>0)-(t<0)||+t}function zi(){if(!this.isValid())return this.localeData().invalidDate();var t,e,i=Ri(this._milliseconds)/1e3,n=Ri(this._days),o=Ri(this._months);t=_(i/60),e=_(t/60),i%=60,t%=60;var s=_(o/12),r=o%=12,a=n,l=e,h=t,d=i?i.toFixed(3).replace(/\.?0+$/,""):"",u=this.asSeconds();if(!u)return"P0D";var c=u<0?"-":"",p=Ui(this._months)!==Ui(u)?"-":"",m=Ui(this._days)!==Ui(u)?"-":"",f=Ui(this._milliseconds)!==Ui(u)?"-":"";return c+"P"+(s?p+s+"Y":"")+(r?p+r+"M":"")+(a?m+a+"D":"")+(l||h||d?"T":"")+(l?f+l+"H":"")+(h?f+h+"M":"")+(d?f+d+"S":"")}var Vi=Se.prototype;return Vi.isValid=function(){return this._isValid},Vi.abs=function(){var t=this._data;return this._milliseconds=gi(this._milliseconds),this._days=gi(this._days),this._months=gi(this._months),t.milliseconds=gi(t.milliseconds),t.seconds=gi(t.seconds),t.minutes=gi(t.minutes),t.hours=gi(t.hours),t.months=gi(t.months),t.years=gi(t.years),this},Vi.add=function(t,e){return vi(this,t,e,1)},Vi.subtract=function(t,e){return vi(this,t,e,-1)},Vi.as=function(t){if(!this.isValid())return NaN;var e,i,n=this._milliseconds;if("month"===(t=B(t))||"quarter"===t||"year"===t)switch(e=this._days+n/864e5,i=this._months+yi(e),t){case"month":return i;case"quarter":return i/3;case"year":return i/12}else switch(e=this._days+Math.round(xi(this._months)),t){case"week":return e/7+n/6048e5;case"day":return e+n/864e5;case"hour":return 24*e+n/36e5;case"minute":return 1440*e+n/6e4;case"second":return 86400*e+n/1e3;case"millisecond":return Math.floor(864e5*e)+n;default:throw new Error("Unknown unit "+t)}},Vi.asMilliseconds=_i,Vi.asSeconds=ki,Vi.asMinutes=Di,Vi.asHours=Ii,Vi.asDays=Ci,Vi.asWeeks=Li,Vi.asMonths=Gi,Vi.asQuarters=Mi,Vi.asYears=Si,Vi.valueOf=function(){return this.isValid()?this._milliseconds+864e5*this._days+this._months%12*2592e6+31536e6*k(this._months/12):NaN},Vi._bubble=function(){var t,e,i,n,o,s=this._milliseconds,r=this._days,a=this._months,l=this._data;return s>=0&&r>=0&&a>=0||s<=0&&r<=0&&a<=0||(s+=864e5*bi(xi(a)+r),r=0,a=0),l.milliseconds=s%1e3,t=_(s/1e3),l.seconds=t%60,e=_(t/60),l.minutes=e%60,i=_(e/60),l.hours=i%24,r+=_(i/24),o=_(yi(r)),a+=o,r-=bi(xi(o)),n=_(a/12),a%=12,l.days=r,l.months=a,l.years=n,this},Vi.clone=function(){return Ye(this)},Vi.get=function(t){return t=B(t),this.isValid()?this[t+"s"]():NaN},Vi.milliseconds=Ni,Vi.seconds=Fi,Vi.minutes=Zi,Vi.hours=Bi,Vi.days=Oi,Vi.weeks=function(){return _(this.days()/7)},Vi.months=Wi,Vi.years=Ai,Vi.humanize=function(t){if(!this.isValid())return this.localeData().invalidDate();var e=this.localeData(),i=Ei(this,!t,e);return t&&(i=e.pastFuture(+this,i)),e.postformat(i)},Vi.toISOString=zi,Vi.toString=zi,Vi.toJSON=zi,Vi.locale=Pe,Vi.localeData=Ke,Vi.toIsoString=C("toIsoString() is deprecated. Please use toISOString() instead (notice the capitals)",zi),Vi.lang=Je,U("X",0,0,"unix"),U("x",0,0,"valueOf"),dt("x",st),dt("X",/[+-]?\d+(\.\d{1,3})?/),mt("X",(function(t,e,i){i._d=new Date(1e3*parseFloat(t,10))})),mt("x",(function(t,e,i){i._d=new Date(k(t))})),o.version="2.24.0",e=Ie,o.fn=di,o.min=function(){var t=[].slice.call(arguments,0);return Ge("isBefore",t)},o.max=function(){var t=[].slice.call(arguments,0);return Ge("isAfter",t)},o.now=function(){return Date.now?Date.now():+new Date},o.utc=p,o.unix=function(t){return Ie(1e3*t)},o.months=function(t,e){return mi(t,e,"months")},o.isDate=h,o.locale=se,o.invalid=g,o.duration=Ye,o.isMoment=w,o.weekdays=function(t,e,i){return fi(t,e,i,"weekdays")},o.parseZone=function(){return Ie.apply(null,arguments).parseZone()},o.localeData=ae,o.isDuration=Qe,o.monthsShort=function(t,e){return mi(t,e,"monthsShort")},o.weekdaysMin=function(t,e,i){return fi(t,e,i,"weekdaysMin")},o.defineLocale=re,o.updateLocale=function(t,e){if(null!=e){var i,n,o=te;null!=(n=oe(t))&&(o=n._config),(i=new N(e=Q(o,e))).parentLocale=ee[t],ee[t]=i,se(t)}else null!=ee[t]&&(null!=ee[t].parentLocale?ee[t]=ee[t].parentLocale:null!=ee[t]&&delete ee[t]);return ee[t]},o.locales=function(){return L(ee)},o.weekdaysShort=function(t,e,i){return fi(t,e,i,"weekdaysShort")},o.normalizeUnits=B,o.relativeTimeRounding=function(t){return void 0===t?Xi:"function"==typeof t&&(Xi=t,!0)},o.relativeTimeThreshold=function(t,e){return void 0!==Ti[t]&&(void 0===e?Ti[t]:(Ti[t]=e,"s"===t&&(Ti.ss=e-1),!0))},o.calendarFormat=function(t,e){var i=t.diff(e,"days",!0);return i<-6?"sameElse":i<-1?"lastWeek":i<0?"lastDay":i<1?"sameDay":i<2?"nextDay":i<7?"nextWeek":"sameElse"},o.prototype=di,o.HTML5_FMT={DATETIME_LOCAL:"YYYY-MM-DDTHH:mm",DATETIME_LOCAL_SECONDS:"YYYY-MM-DDTHH:mm:ss",DATETIME_LOCAL_MS:"YYYY-MM-DDTHH:mm:ss.SSS",DATE:"YYYY-MM-DD",TIME:"HH:mm",TIME_SECONDS:"HH:mm:ss",TIME_MS:"HH:mm:ss.SSS",WEEK:"GGGG-[W]WW",MONTH:"YYYY-MM"},o}()})),r=(n((function(t,e){(function(t){function e(t,e,i,n){var o={m:["eine Minute","einer Minute"],h:["eine Stunde","einer Stunde"],d:["ein Tag","einem Tag"],dd:[t+" Tage",t+" Tagen"],M:["ein Monat","einem Monat"],MM:[t+" Monate",t+" Monaten"],y:["ein Jahr","einem Jahr"],yy:[t+" Jahre",t+" Jahren"]};return e?o[i][0]:o[i][1]}t.defineLocale("de",{months:"Januar_Februar_März_April_Mai_Juni_Juli_August_September_Oktober_November_Dezember".split("_"),monthsShort:"Jan._Feb._März_Apr._Mai_Juni_Juli_Aug._Sep._Okt._Nov._Dez.".split("_"),monthsParseExact:!0,weekdays:"Sonntag_Montag_Dienstag_Mittwoch_Donnerstag_Freitag_Samstag".split("_"),weekdaysShort:"So._Mo._Di._Mi._Do._Fr._Sa.".split("_"),weekdaysMin:"So_Mo_Di_Mi_Do_Fr_Sa".split("_"),weekdaysParseExact:!0,longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD.MM.YYYY",LL:"D. MMMM YYYY",LLL:"D. MMMM YYYY HH:mm",LLLL:"dddd, D. MMMM YYYY HH:mm"},calendar:{sameDay:"[heute um] LT [Uhr]",sameElse:"L",nextDay:"[morgen um] LT [Uhr]",nextWeek:"dddd [um] LT [Uhr]",lastDay:"[gestern um] LT [Uhr]",lastWeek:"[letzten] dddd [um] LT [Uhr]"},relativeTime:{future:"in %s",past:"vor %s",s:"ein paar Sekunden",ss:"%d Sekunden",m:e,mm:"%d Minuten",h:e,hh:"%d Stunden",d:e,dd:e,M:e,MM:e,y:e,yy:e},dayOfMonthOrdinalParse:/\d{1,2}\./,ordinal:"%d.",week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){var e="ene._feb._mar._abr._may._jun._jul._ago._sep._oct._nov._dic.".split("_"),i="ene_feb_mar_abr_may_jun_jul_ago_sep_oct_nov_dic".split("_"),n=[/^ene/i,/^feb/i,/^mar/i,/^abr/i,/^may/i,/^jun/i,/^jul/i,/^ago/i,/^sep/i,/^oct/i,/^nov/i,/^dic/i],o=/^(enero|febrero|marzo|abril|mayo|junio|julio|agosto|septiembre|octubre|noviembre|diciembre|ene\.?|feb\.?|mar\.?|abr\.?|may\.?|jun\.?|jul\.?|ago\.?|sep\.?|oct\.?|nov\.?|dic\.?)/i;t.defineLocale("es",{months:"enero_febrero_marzo_abril_mayo_junio_julio_agosto_septiembre_octubre_noviembre_diciembre".split("_"),monthsShort:function(t,n){return t?/-MMM-/.test(n)?i[t.month()]:e[t.month()]:e},monthsRegex:o,monthsShortRegex:o,monthsStrictRegex:/^(enero|febrero|marzo|abril|mayo|junio|julio|agosto|septiembre|octubre|noviembre|diciembre)/i,monthsShortStrictRegex:/^(ene\.?|feb\.?|mar\.?|abr\.?|may\.?|jun\.?|jul\.?|ago\.?|sep\.?|oct\.?|nov\.?|dic\.?)/i,monthsParse:n,longMonthsParse:n,shortMonthsParse:n,weekdays:"domingo_lunes_martes_miércoles_jueves_viernes_sábado".split("_"),weekdaysShort:"dom._lun._mar._mié._jue._vie._sáb.".split("_"),weekdaysMin:"do_lu_ma_mi_ju_vi_sá".split("_"),weekdaysParseExact:!0,longDateFormat:{LT:"H:mm",LTS:"H:mm:ss",L:"DD/MM/YYYY",LL:"D [de] MMMM [de] YYYY",LLL:"D [de] MMMM [de] YYYY H:mm",LLLL:"dddd, D [de] MMMM [de] YYYY H:mm"},calendar:{sameDay:function(){return"[hoy a la"+(1!==this.hours()?"s":"")+"] LT"},nextDay:function(){return"[mañana a la"+(1!==this.hours()?"s":"")+"] LT"},nextWeek:function(){return"dddd [a la"+(1!==this.hours()?"s":"")+"] LT"},lastDay:function(){return"[ayer a la"+(1!==this.hours()?"s":"")+"] LT"},lastWeek:function(){return"[el] dddd [pasado a la"+(1!==this.hours()?"s":"")+"] LT"},sameElse:"L"},relativeTime:{future:"en %s",past:"hace %s",s:"unos segundos",ss:"%d segundos",m:"un minuto",mm:"%d minutos",h:"una hora",hh:"%d horas",d:"un día",dd:"%d días",M:"un mes",MM:"%d meses",y:"un año",yy:"%d años"},dayOfMonthOrdinalParse:/\d{1,2}º/,ordinal:"%dº",week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){t.defineLocale("fr",{months:"janvier_février_mars_avril_mai_juin_juillet_août_septembre_octobre_novembre_décembre".split("_"),monthsShort:"janv._févr._mars_avr._mai_juin_juil._août_sept._oct._nov._déc.".split("_"),monthsParseExact:!0,weekdays:"dimanche_lundi_mardi_mercredi_jeudi_vendredi_samedi".split("_"),weekdaysShort:"dim._lun._mar._mer._jeu._ven._sam.".split("_"),weekdaysMin:"di_lu_ma_me_je_ve_sa".split("_"),weekdaysParseExact:!0,longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD/MM/YYYY",LL:"D MMMM YYYY",LLL:"D MMMM YYYY HH:mm",LLLL:"dddd D MMMM YYYY HH:mm"},calendar:{sameDay:"[Aujourd’hui à] LT",nextDay:"[Demain à] LT",nextWeek:"dddd [à] LT",lastDay:"[Hier à] LT",lastWeek:"dddd [dernier à] LT",sameElse:"L"},relativeTime:{future:"dans %s",past:"il y a %s",s:"quelques secondes",ss:"%d secondes",m:"une minute",mm:"%d minutes",h:"une heure",hh:"%d heures",d:"un jour",dd:"%d jours",M:"un mois",MM:"%d mois",y:"un an",yy:"%d ans"},dayOfMonthOrdinalParse:/\d{1,2}(er|)/,ordinal:function(t,e){switch(e){case"D":return t+(1===t?"er":"");default:case"M":case"Q":case"DDD":case"d":return t+(1===t?"er":"e");case"w":case"W":return t+(1===t?"re":"e")}},week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){t.defineLocale("it",{months:"gennaio_febbraio_marzo_aprile_maggio_giugno_luglio_agosto_settembre_ottobre_novembre_dicembre".split("_"),monthsShort:"gen_feb_mar_apr_mag_giu_lug_ago_set_ott_nov_dic".split("_"),weekdays:"domenica_lunedì_martedì_mercoledì_giovedì_venerdì_sabato".split("_"),weekdaysShort:"dom_lun_mar_mer_gio_ven_sab".split("_"),weekdaysMin:"do_lu_ma_me_gi_ve_sa".split("_"),longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD/MM/YYYY",LL:"D MMMM YYYY",LLL:"D MMMM YYYY HH:mm",LLLL:"dddd D MMMM YYYY HH:mm"},calendar:{sameDay:"[Oggi alle] LT",nextDay:"[Domani alle] LT",nextWeek:"dddd [alle] LT",lastDay:"[Ieri alle] LT",lastWeek:function(){switch(this.day()){case 0:return"[la scorsa] dddd [alle] LT";default:return"[lo scorso] dddd [alle] LT"}},sameElse:"L"},relativeTime:{future:function(t){return(/^[0-9].+$/.test(t)?"tra":"in")+" "+t},past:"%s fa",s:"alcuni secondi",ss:"%d secondi",m:"un minuto",mm:"%d minuti",h:"un'ora",hh:"%d ore",d:"un giorno",dd:"%d giorni",M:"un mese",MM:"%d mesi",y:"un anno",yy:"%d anni"},dayOfMonthOrdinalParse:/\d{1,2}º/,ordinal:"%dº",week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){t.defineLocale("ja",{months:"一月_二月_三月_四月_五月_六月_七月_八月_九月_十月_十一月_十二月".split("_"),monthsShort:"1月_2月_3月_4月_5月_6月_7月_8月_9月_10月_11月_12月".split("_"),weekdays:"日曜日_月曜日_火曜日_水曜日_木曜日_金曜日_土曜日".split("_"),weekdaysShort:"日_月_火_水_木_金_土".split("_"),weekdaysMin:"日_月_火_水_木_金_土".split("_"),longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"YYYY/MM/DD",LL:"YYYY年M月D日",LLL:"YYYY年M月D日 HH:mm",LLLL:"YYYY年M月D日 dddd HH:mm",l:"YYYY/MM/DD",ll:"YYYY年M月D日",lll:"YYYY年M月D日 HH:mm",llll:"YYYY年M月D日(ddd) HH:mm"},meridiemParse:/午前|午後/i,isPM:function(t){return"午後"===t},meridiem:function(t,e,i){return t<12?"午前":"午後"},calendar:{sameDay:"[今日] LT",nextDay:"[明日] LT",nextWeek:function(t){return t.week()=20?"ste":"de")},week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){var e="styczeń_luty_marzec_kwiecień_maj_czerwiec_lipiec_sierpień_wrzesień_październik_listopad_grudzień".split("_"),i="stycznia_lutego_marca_kwietnia_maja_czerwca_lipca_sierpnia_września_października_listopada_grudnia".split("_");function n(t){return t%10<5&&t%10>1&&~~(t/10)%10!=1}function o(t,e,i){var o=t+" ";switch(i){case"ss":return o+(n(t)?"sekundy":"sekund");case"m":return e?"minuta":"minutę";case"mm":return o+(n(t)?"minuty":"minut");case"h":return e?"godzina":"godzinę";case"hh":return o+(n(t)?"godziny":"godzin");case"MM":return o+(n(t)?"miesiące":"miesięcy");case"yy":return o+(n(t)?"lata":"lat")}}t.defineLocale("pl",{months:function(t,n){return t?""===n?"("+i[t.month()]+"|"+e[t.month()]+")":/D MMMM/.test(n)?i[t.month()]:e[t.month()]:e},monthsShort:"sty_lut_mar_kwi_maj_cze_lip_sie_wrz_paź_lis_gru".split("_"),weekdays:"niedziela_poniedziałek_wtorek_środa_czwartek_piątek_sobota".split("_"),weekdaysShort:"ndz_pon_wt_śr_czw_pt_sob".split("_"),weekdaysMin:"Nd_Pn_Wt_Śr_Cz_Pt_So".split("_"),longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD.MM.YYYY",LL:"D MMMM YYYY",LLL:"D MMMM YYYY HH:mm",LLLL:"dddd, D MMMM YYYY HH:mm"},calendar:{sameDay:"[Dziś o] LT",nextDay:"[Jutro o] LT",nextWeek:function(){switch(this.day()){case 0:return"[W niedzielę o] LT";case 2:return"[We wtorek o] LT";case 3:return"[W środę o] LT";case 6:return"[W sobotę o] LT";default:return"[W] dddd [o] LT"}},lastDay:"[Wczoraj o] LT",lastWeek:function(){switch(this.day()){case 0:return"[W zeszłą niedzielę o] LT";case 3:return"[W zeszłą środę o] LT";case 6:return"[W zeszłą sobotę o] LT";default:return"[W zeszły] dddd [o] LT"}},sameElse:"L"},relativeTime:{future:"za %s",past:"%s temu",s:"kilka sekund",ss:o,m:o,mm:o,h:o,hh:o,d:"1 dzień",dd:"%d dni",M:"miesiąc",MM:o,y:"rok",yy:o},dayOfMonthOrdinalParse:/\d{1,2}\./,ordinal:"%d.",week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){function e(t,e,i){var n,o;return"m"===i?e?"минута":"минуту":t+" "+(n=+t,o={ss:e?"секунда_секунды_секунд":"секунду_секунды_секунд",mm:e?"минута_минуты_минут":"минуту_минуты_минут",hh:"час_часа_часов",dd:"день_дня_дней",MM:"месяц_месяца_месяцев",yy:"год_года_лет"}[i].split("_"),n%10==1&&n%100!=11?o[0]:n%10>=2&&n%10<=4&&(n%100<10||n%100>=20)?o[1]:o[2])}var i=[/^янв/i,/^фев/i,/^мар/i,/^апр/i,/^ма[йя]/i,/^июн/i,/^июл/i,/^авг/i,/^сен/i,/^окт/i,/^ноя/i,/^дек/i];t.defineLocale("ru",{months:{format:"января_февраля_марта_апреля_мая_июня_июля_августа_сентября_октября_ноября_декабря".split("_"),standalone:"январь_февраль_март_апрель_май_июнь_июль_август_сентябрь_октябрь_ноябрь_декабрь".split("_")},monthsShort:{format:"янв._февр._мар._апр._мая_июня_июля_авг._сент._окт._нояб._дек.".split("_"),standalone:"янв._февр._март_апр._май_июнь_июль_авг._сент._окт._нояб._дек.".split("_")},weekdays:{standalone:"воскресенье_понедельник_вторник_среда_четверг_пятница_суббота".split("_"),format:"воскресенье_понедельник_вторник_среду_четверг_пятницу_субботу".split("_"),isFormat:/\[ ?[Вв] ?(?:прошлую|следующую|эту)? ?\] ?dddd/},weekdaysShort:"вс_пн_вт_ср_чт_пт_сб".split("_"),weekdaysMin:"вс_пн_вт_ср_чт_пт_сб".split("_"),monthsParse:i,longMonthsParse:i,shortMonthsParse:i,monthsRegex:/^(январ[ья]|янв\.?|феврал[ья]|февр?\.?|марта?|мар\.?|апрел[ья]|апр\.?|ма[йя]|июн[ья]|июн\.?|июл[ья]|июл\.?|августа?|авг\.?|сентябр[ья]|сент?\.?|октябр[ья]|окт\.?|ноябр[ья]|нояб?\.?|декабр[ья]|дек\.?)/i,monthsShortRegex:/^(январ[ья]|янв\.?|феврал[ья]|февр?\.?|марта?|мар\.?|апрел[ья]|апр\.?|ма[йя]|июн[ья]|июн\.?|июл[ья]|июл\.?|августа?|авг\.?|сентябр[ья]|сент?\.?|октябр[ья]|окт\.?|ноябр[ья]|нояб?\.?|декабр[ья]|дек\.?)/i,monthsStrictRegex:/^(январ[яь]|феврал[яь]|марта?|апрел[яь]|ма[яй]|июн[яь]|июл[яь]|августа?|сентябр[яь]|октябр[яь]|ноябр[яь]|декабр[яь])/i,monthsShortStrictRegex:/^(янв\.|февр?\.|мар[т.]|апр\.|ма[яй]|июн[ья.]|июл[ья.]|авг\.|сент?\.|окт\.|нояб?\.|дек\.)/i,longDateFormat:{LT:"H:mm",LTS:"H:mm:ss",L:"DD.MM.YYYY",LL:"D MMMM YYYY г.",LLL:"D MMMM YYYY г., H:mm",LLLL:"dddd, D MMMM YYYY г., H:mm"},calendar:{sameDay:"[Сегодня, в] LT",nextDay:"[Завтра, в] LT",lastDay:"[Вчера, в] LT",nextWeek:function(t){if(t.week()===this.week())return 2===this.day()?"[Во] dddd, [в] LT":"[В] dddd, [в] LT";switch(this.day()){case 0:return"[В следующее] dddd, [в] LT";case 1:case 2:case 4:return"[В следующий] dddd, [в] LT";case 3:case 5:case 6:return"[В следующую] dddd, [в] LT"}},lastWeek:function(t){if(t.week()===this.week())return 2===this.day()?"[Во] dddd, [в] LT":"[В] dddd, [в] LT";switch(this.day()){case 0:return"[В прошлое] dddd, [в] LT";case 1:case 2:case 4:return"[В прошлый] dddd, [в] LT";case 3:case 5:case 6:return"[В прошлую] dddd, [в] LT"}},sameElse:"L"},relativeTime:{future:"через %s",past:"%s назад",s:"несколько секунд",ss:e,m:e,mm:e,h:"час",hh:e,d:"день",dd:e,M:"месяц",MM:e,y:"год",yy:e},meridiemParse:/ночи|утра|дня|вечера/i,isPM:function(t){return/^(дня|вечера)$/.test(t)},meridiem:function(t,e,i){return t<4?"ночи":t<12?"утра":t<17?"дня":"вечера"},dayOfMonthOrdinalParse:/\d{1,2}-(й|го|я)/,ordinal:function(t,e){switch(e){case"M":case"d":case"DDD":return t+"-й";case"D":return t+"-го";case"w":case"W":return t+"-я";default:return t}},week:{dow:1,doy:4}})})(s)})),n((function(t,e){(function(t){function e(t,e,i){var n,o;return"m"===i?e?"хвилина":"хвилину":"h"===i?e?"година":"годину":t+" "+(n=+t,o={ss:e?"секунда_секунди_секунд":"секунду_секунди_секунд",mm:e?"хвилина_хвилини_хвилин":"хвилину_хвилини_хвилин",hh:e?"година_години_годин":"годину_години_годин",dd:"день_дні_днів",MM:"місяць_місяці_місяців",yy:"рік_роки_років"}[i].split("_"),n%10==1&&n%100!=11?o[0]:n%10>=2&&n%10<=4&&(n%100<10||n%100>=20)?o[1]:o[2])}function i(t){return function(){return t+"о"+(11===this.hours()?"б":"")+"] LT"}}t.defineLocale("uk",{months:{format:"січня_лютого_березня_квітня_травня_червня_липня_серпня_вересня_жовтня_листопада_грудня".split("_"),standalone:"січень_лютий_березень_квітень_травень_червень_липень_серпень_вересень_жовтень_листопад_грудень".split("_")},monthsShort:"січ_лют_бер_квіт_трав_черв_лип_серп_вер_жовт_лист_груд".split("_"),weekdays:function(t,e){var i={nominative:"неділя_понеділок_вівторок_середа_четвер_п’ятниця_субота".split("_"),accusative:"неділю_понеділок_вівторок_середу_четвер_п’ятницю_суботу".split("_"),genitive:"неділі_понеділка_вівторка_середи_четверга_п’ятниці_суботи".split("_")};return!0===t?i.nominative.slice(1,7).concat(i.nominative.slice(0,1)):t?i[/(\[[ВвУу]\]) ?dddd/.test(e)?"accusative":/\[?(?:минулої|наступної)? ?\] ?dddd/.test(e)?"genitive":"nominative"][t.day()]:i.nominative},weekdaysShort:"нд_пн_вт_ср_чт_пт_сб".split("_"),weekdaysMin:"нд_пн_вт_ср_чт_пт_сб".split("_"),longDateFormat:{LT:"HH:mm",LTS:"HH:mm:ss",L:"DD.MM.YYYY",LL:"D MMMM YYYY р.",LLL:"D MMMM YYYY р., HH:mm",LLLL:"dddd, D MMMM YYYY р., HH:mm"},calendar:{sameDay:i("[Сьогодні "),nextDay:i("[Завтра "),lastDay:i("[Вчора "),nextWeek:i("[У] dddd ["),lastWeek:function(){switch(this.day()){case 0:case 3:case 5:case 6:return i("[Минулої] dddd [").call(this);case 1:case 2:case 4:return i("[Минулого] dddd [").call(this)}},sameElse:"L"},relativeTime:{future:"за %s",past:"%s тому",s:"декілька секунд",ss:e,m:e,mm:e,h:"годину",hh:e,d:"день",dd:e,M:"місяць",MM:e,y:"рік",yy:e},meridiemParse:/ночі|ранку|дня|вечора/,isPM:function(t){return/^(дня|вечора)$/.test(t)},meridiem:function(t,e,i){return t<4?"ночі":t<12?"ранку":t<17?"дня":"вечора"},dayOfMonthOrdinalParse:/\d{1,2}-(й|го)/,ordinal:function(t,e){switch(e){case"M":case"d":case"DDD":case"w":case"W":return t+"-й";case"D":return t+"-го";default:return t}},week:{dow:1,doy:7}})})(s)})),function(t){return t&&t.Math==Math&&t}),a=r("object"==typeof globalThis&&globalThis)||r("object"==typeof window&&window)||r("object"==typeof self&&self)||r("object"==typeof e&&e)||Function("return this")(),l=function(t){try{return!!t()}catch(t){return!0}},h=!l((function(){return 7!=Object.defineProperty({},1,{get:function(){return 7}})[1]})),d={}.propertyIsEnumerable,u=Object.getOwnPropertyDescriptor,c={f:u&&!d.call({1:2},1)?function(t){var e=u(this,t);return!!e&&e.enumerable}:d},p=function(t,e){return{enumerable:!(1&t),configurable:!(2&t),writable:!(4&t),value:e}},m={}.toString,f=function(t){return m.call(t).slice(8,-1)},g="".split,v=l((function(){return!Object("z").propertyIsEnumerable(0)}))?function(t){return"String"==f(t)?g.call(t,""):Object(t)}:Object,b=function(t){if(null==t)throw TypeError("Can't call method on "+t);return t},y=function(t){return v(b(t))},x=function(t){return"object"==typeof t?null!==t:"function"==typeof t},w=function(t,e){if(!x(t))return t;var i,n;if(e&&"function"==typeof(i=t.toString)&&!x(n=i.call(t)))return n;if("function"==typeof(i=t.valueOf)&&!x(n=i.call(t)))return n;if(!e&&"function"==typeof(i=t.toString)&&!x(n=i.call(t)))return n;throw TypeError("Can't convert object to primitive value")},_={}.hasOwnProperty,k=function(t,e){return _.call(t,e)},D=a.document,I=x(D)&&x(D.createElement),C=function(t){return I?D.createElement(t):{}},L=!h&&!l((function(){return 7!=Object.defineProperty(C("div"),"a",{get:function(){return 7}}).a})),G=Object.getOwnPropertyDescriptor,M={f:h?G:function(t,e){if(t=y(t),e=w(e,!0),L)try{return G(t,e)}catch(t){}if(k(t,e))return p(!c.f.call(t,e),t[e])}},S=/#|\.prototype\./,Q=function(t,e){var i=F[N(t)];return i==B||i!=Z&&("function"==typeof e?l(e):!!e)},N=Q.normalize=function(t){return String(t).replace(S,".").toLowerCase()},F=Q.data={},Z=Q.NATIVE="N",B=Q.POLYFILL="P",O=Q,W={},A=function(t){if("function"!=typeof t)throw TypeError(String(t)+" is not a function");return t},X=function(t,e,i){if(A(t),void 0===e)return t;switch(i){case 0:return function(){return t.call(e)};case 1:return function(i){return t.call(e,i)};case 2:return function(i,n){return t.call(e,i,n)};case 3:return function(i,n,o){return t.call(e,i,n,o)}}return function(){return t.apply(e,arguments)}},T=function(t){if(!x(t))throw TypeError(String(t)+" is not an object");return t},Y=Object.defineProperty,E={f:h?Y:function(t,e,i){if(T(t),e=w(e,!0),T(i),L)try{return Y(t,e,i)}catch(t){}if("get"in i||"set"in i)throw TypeError("Accessors not supported");return"value"in i&&(t[e]=i.value),t}},R=h?function(t,e,i){return E.f(t,e,p(1,i))}:function(t,e,i){return t[e]=i,t},U=M.f,z=function(t){var e=function(e,i,n){if(this instanceof t){switch(arguments.length){case 0:return new t;case 1:return new t(e);case 2:return new t(e,i)}return new t(e,i,n)}return t.apply(this,arguments)};return e.prototype=t.prototype,e},V=function(t,e){var i,n,o,s,r,l,h,d,u=t.target,c=t.global,p=t.stat,m=t.proto,f=c?a:p?a[u]:(a[u]||{}).prototype,g=c?W:W[u]||(W[u]={}),v=g.prototype;for(o in e)i=!O(c?o:u+(p?".":"#")+o,t.forced)&&f&&k(f,o),r=g[o],i&&(l=t.noTargetGet?(d=U(f,o))&&d.value:f[o]),s=i&&l?l:e[o],i&&typeof r==typeof s||(h=t.bind&&i?X(s,a):t.wrap&&i?z(s):m&&"function"==typeof s?X(Function.call,s):s,(t.sham||s&&s.sham||r&&r.sham)&&R(h,"sham",!0),g[o]=h,m&&(k(W,n=u+"Prototype")||R(W,n,{}),W[n][o]=s,t.real&&v&&!v[o]&&R(v,o,s)))};V({target:"Object",stat:!0,forced:!h,sham:!h},{defineProperty:E.f});var j=n((function(t){var e=W.Object,i=t.exports=function(t,i,n){return e.defineProperty(t,i,n)};e.defineProperty.sham&&(i.sham=!0)})),H=j,P=Math.ceil,J=Math.floor,K=function(t){return isNaN(t=+t)?0:(t>0?J:P)(t)},q=Math.min,$=function(t){return t>0?q(K(t),9007199254740991):0},tt=Math.max,et=Math.min,it=function(t,e){var i=K(t);return i<0?tt(i+e,0):et(i,e)},nt=function(t){return function(e,i,n){var o,s=y(e),r=$(s.length),a=it(n,r);if(t&&i!=i){for(;r>a;)if((o=s[a++])!=o)return!0}else for(;r>a;a++)if((t||a in s)&&s[a]===i)return t||a||0;return!t&&-1}},ot={includes:nt(!0),indexOf:nt(!1)},st={},rt=ot.indexOf,at=function(t,e){var i,n=y(t),o=0,s=[];for(i in n)!k(st,i)&&k(n,i)&&s.push(i);for(;e.length>o;)k(n,i=e[o++])&&(~rt(s,i)||s.push(i));return s},lt=["constructor","hasOwnProperty","isPrototypeOf","propertyIsEnumerable","toLocaleString","toString","valueOf"],ht=Object.keys||function(t){return at(t,lt)},dt=h?Object.defineProperties:function(t,e){T(t);for(var i,n=ht(e),o=n.length,s=0;o>s;)E.f(t,i=n[s++],e[i]);return t};V({target:"Object",stat:!0,forced:!h,sham:!h},{defineProperties:dt});var ut=n((function(t){var e=W.Object,i=t.exports=function(t,i){return e.defineProperties(t,i)};e.defineProperties.sham&&(i.sham=!0)})),ct=function(t){return"function"==typeof t?t:void 0},pt=function(t,e){return arguments.length<2?ct(W[t])||ct(a[t]):W[t]&&W[t][e]||a[t]&&a[t][e]},mt=lt.concat("length","prototype"),ft={f:Object.getOwnPropertyNames||function(t){return at(t,mt)}},gt={f:Object.getOwnPropertySymbols},vt=pt("Reflect","ownKeys")||function(t){var e=ft.f(T(t)),i=gt.f;return i?e.concat(i(t)):e},bt=function(t,e,i){var n=w(e);n in t?E.f(t,n,p(0,i)):t[n]=i};V({target:"Object",stat:!0,sham:!h},{getOwnPropertyDescriptors:function(t){for(var e,i,n=y(t),o=M.f,s=vt(n),r={},a=0;s.length>a;)void 0!==(i=o(n,e=s[a++]))&&bt(r,e,i);return r}});var yt=W.Object.getOwnPropertyDescriptors,xt=M.f,wt=l((function(){xt(1)}));V({target:"Object",stat:!0,forced:!h||wt,sham:!h},{getOwnPropertyDescriptor:function(t,e){return xt(y(t),e)}});var _t,kt=n((function(t){var e=W.Object,i=t.exports=function(t,i){return e.getOwnPropertyDescriptor(t,i)};e.getOwnPropertyDescriptor.sham&&(i.sham=!0)})),Dt=!!Object.getOwnPropertySymbols&&!l((function(){return!String(Symbol())})),It=Dt&&!Symbol.sham&&"symbol"==typeof Symbol.iterator,Ct=Array.isArray||function(t){return"Array"==f(t)},Lt=function(t){return Object(b(t))},Gt=pt("document","documentElement"),Mt=a["__core-js_shared__"]||function(t,e){try{R(a,t,e)}catch(i){a[t]=e}return e}("__core-js_shared__",{}),St=n((function(t){(t.exports=function(t,e){return Mt[t]||(Mt[t]=void 0!==e?e:{})})("versions",[]).push({version:"3.6.4",mode:"pure",copyright:"© 2020 Denis Pushkarev (zloirock.ru)"})})),Qt=0,Nt=Math.random(),Ft=function(t){return"Symbol("+String(void 0===t?"":t)+")_"+(++Qt+Nt).toString(36)},Zt=St("keys"),Bt=function(t){return Zt[t]||(Zt[t]=Ft(t))},Ot=Bt("IE_PROTO"),Wt=function(){},At=function(t){return" - + diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 3fa4eccbfacd1..68c8d8260fe3c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -442,15 +442,10 @@ private[ui] class JobDataSource( } private def jobRow(jobData: v1.JobData): JobTableRowData = { - val duration: Option[Long] = { - jobData.submissionTime.map { start => - val end = jobData.completionTime.map(_.getTime()).getOrElse(System.currentTimeMillis()) - end - start.getTime() - } - } - val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") + val duration: Option[Long] = JobDataUtil.getDuration(jobData) + val formattedDuration = JobDataUtil.getFormattedDuration(jobData) val submissionTime = jobData.submissionTime - val formattedSubmissionTime = submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") + val formattedSubmissionTime = JobDataUtil.getFormattedSubmissionTime(jobData) val (lastStageName, lastStageDescription) = lastStageNameAndDescription(store, jobData) val jobDescription = diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobDataUtil.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobDataUtil.scala new file mode 100644 index 0000000000000..f357fec0ab906 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobDataUtil.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ui.jobs + +import org.apache.spark.status.api.v1.JobData +import org.apache.spark.ui.UIUtils + +private[ui] object JobDataUtil { + def getDuration(jobData: JobData): Option[Long] = { + jobData.submissionTime.map { start => + val end = jobData.completionTime.map(_.getTime()).getOrElse(System.currentTimeMillis()) + end - start.getTime() + } + } + + def getFormattedDuration(jobData: JobData): String = { + val duration = getDuration(jobData) + duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") + } + + def getFormattedSubmissionTime(jobData: JobData): String = { + jobData.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 348baab23b2ba..eacc6ce527567 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -314,6 +314,14 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP Status: {jobData.status} +
  • + Submitted: + {JobDataUtil.getFormattedSubmissionTime(jobData)} +
  • +
  • + Duration: + {JobDataUtil.getFormattedDuration(jobData)} +
  • { if (sqlExecutionId.isDefined) {
  • diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 9f332ba608082..c7db2127a6f04 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -95,7 +95,7 @@ private[spark] object Utils extends Logging { */ val DEFAULT_DRIVER_MEM_MB = JavaUtils.DEFAULT_DRIVER_MEM_MB.toInt - val MAX_DIR_CREATION_ATTEMPTS: Int = 10 + private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @volatile private var localRootDirs: Array[String] = null /** Scheme used for files that are locally available on worker nodes in the cluster. */ diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 8fa33f4915ea4..807f0eb808f9b 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable import org.mockito.ArgumentMatchers.{any, eq => meq} -import org.mockito.Mockito.{mock, never, verify, when} +import org.mockito.Mockito.{mock, never, times, verify, when} import org.scalatest.PrivateMethodTester import org.apache.spark.executor.ExecutorMetrics @@ -265,6 +265,26 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(numExecutorsTarget(manager, rprof1.id) === 10) } + test("add executors multiple profiles initial num same as needed") { + // test when the initial number of executors equals the number needed for the first + // stage using a non default profile to make sure we request the intitial number + // properly. Here initial is 2, each executor in ResourceProfile 1 can have 2 tasks + // per executor, and start a stage with 4 tasks, which would need 2 executors. + val clock = new ManualClock(8888L) + val manager = createManager(createConf(0, 10, 2), clock) + val rp1 = new ResourceProfileBuilder() + val execReqs = new ExecutorResourceRequests().cores(2).resource("gpu", 2) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + rp1.require(execReqs).require(taskReqs) + val rprof1 = rp1.build + rpManager.addResourceProfile(rprof1) + when(client.requestTotalExecutors(any(), any(), any())).thenReturn(true) + post(SparkListenerStageSubmitted(createStageInfo(1, 4, rp = rprof1))) + // called once on start and a second time on stage submit with initial number + verify(client, times(2)).requestTotalExecutors(any(), any(), any()) + assert(numExecutorsTarget(manager, rprof1.id) === 2) + } + test("remove executors multiple profiles") { val manager = createManager(createConf(5, 10, 5)) val rp1 = new ResourceProfileBuilder() @@ -841,7 +861,10 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { } test ("interleaving add and remove") { - val manager = createManager(createConf(5, 12, 5)) + // use ManualClock to disable ExecutorAllocationManager.schedule() + // in order to avoid unexpected update of target executors + val clock = new ManualClock() + val manager = createManager(createConf(5, 12, 5), clock) post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) val updatesNeeded = diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 9d4736825618e..1f3243400a918 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -429,6 +429,7 @@ class SparkSubmitSuite test("handles k8s cluster mode") { val clArgs = Seq( "--deploy-mode", "cluster", + "--proxy-user", "test.user", "--master", "k8s://host:port", "--executor-memory", "5g", "--class", "org.SomeClass", @@ -444,6 +445,7 @@ class SparkSubmitSuite childArgsMap.get("--primary-java-resource") should be (Some("file:/home/thejar.jar")) childArgsMap.get("--main-class") should be (Some("org.SomeClass")) childArgsMap.get("--arg") should be (Some("arg1")) + childArgsMap.get("--proxy-user") should be (Some("test.user")) mainClass should be (KUBERNETES_CLUSTER_SUBMIT_CLASS) classpath should have length (0) conf.get("spark.master") should be ("k8s://https://host:port") diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index f8b99302c4ad5..57cbda3c0620d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -510,8 +510,7 @@ class StandaloneDynamicAllocationSuite val taskScheduler = mock(classOf[TaskSchedulerImpl]) when(taskScheduler.nodeBlacklist()).thenReturn(Set("blacklisted-host")) - when(taskScheduler.resourceOffers(any())).thenReturn(Nil) - when(taskScheduler.resourcesReqsPerTask).thenReturn(Seq.empty) + when(taskScheduler.resourceOffers(any(), any[Boolean])).thenReturn(Nil) when(taskScheduler.sc).thenReturn(sc) val rpcEnv = RpcEnv.create("test-rpcenv", "localhost", 0, conf, securityManager) diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 3134a738b33fa..e0b586074b89e 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -301,8 +301,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val taskId = 1000000 // We don't really verify the data, just pass it around. val data = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) - val taskDescription = new TaskDescription(taskId, 2, "1", "TASK 1000000", - 19, 1, mutable.Map.empty, mutable.Map.empty, new Properties, + val taskDescription = new TaskDescription(taskId, 2, "1", "TASK 1000000", 19, + 1, mutable.Map.empty, mutable.Map.empty, new Properties, Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), data) val serializedTaskDescription = TaskDescription.encode(taskDescription) backend.executor = mock[Executor] 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 edddf88a28f85..c804102e4ab2c 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 @@ -105,7 +105,7 @@ class NettyBlockTransferServiceSuite // This is used to touch an IOException during fetching block. when(client.sendRpc(any(), any())).thenAnswer(_ => {throw new IOException()}) var createClientCount = 0 - when(clientFactory.createClient(any(), any())).thenAnswer(_ => { + when(clientFactory.createClient(any(), any(), any())).thenAnswer(_ => { createClientCount += 1 client }) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDCleanerSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDCleanerSuite.scala new file mode 100644 index 0000000000000..05240f95e616d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/RDDCleanerSuite.scala @@ -0,0 +1,77 @@ +/* + * 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.rdd + +import java.io.File + +import scala.collection.JavaConverters._ + +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.TrueFileFilter +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark._ +import org.apache.spark.util.Utils + + +class RDDCleanerSuite extends SparkFunSuite with BeforeAndAfterEach { + override def beforeEach(): Unit = { + super.beforeEach() + // Once `Utils.getOrCreateLocalRootDirs` is called, it is cached in `Utils.localRootDirs`. + // Unless this is manually cleared before and after a test, it returns the same directory + // set before even if 'spark.local.dir' is configured afterwards. + Utils.clearLocalRootDirs() + } + + override def afterEach(): Unit = { + Utils.clearLocalRootDirs() + super.afterEach() + } + + test("RDD shuffle cleanup standalone") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "test", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Test checkpoint and clean parents + val input = sc.parallelize(1 to 1000) + val keyed = input.map(x => (x % 20, 1)) + val shuffled = keyed.reduceByKey(_ + _) + val keysOnly = shuffled.keys + keysOnly.count() + assert(getAllFiles.size > 0) + keysOnly.cleanShuffleDependencies(true) + val resultingFiles = getAllFiles + assert(resultingFiles === Set()) + // Ensure running count again works fine even if we kill the shuffle files. + assert(keysOnly.count() === 20) + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala index 075260317284d..004618a161b44 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -98,6 +98,36 @@ class ResourceProfileManagerSuite extends SparkFunSuite { assert(error.contains("ResourceProfiles are only supported on YARN with dynamic allocation")) } + test("ResourceProfileManager has equivalent profile") { + val conf = new SparkConf().set(EXECUTOR_CORES, 4) + val rpmanager = new ResourceProfileManager(conf) + var rpAlreadyExist: Option[ResourceProfile] = None + val checkId = 500 + for (i <- 1 to 1000) { + val rprofBuilder = new ResourceProfileBuilder() + val ereqs = new ExecutorResourceRequests() + ereqs.cores(i).memory("4g").memoryOverhead("2000m") + val treqs = new TaskResourceRequests() + treqs.cpus(i) + rprofBuilder.require(ereqs).require(treqs) + val rprof = rprofBuilder.build + rpmanager.addResourceProfile(rprof) + if (i == checkId) rpAlreadyExist = Some(rprof) + } + val rpNotMatch = new ResourceProfileBuilder().build + assert(rpmanager.getEquivalentProfile(rpNotMatch).isEmpty, + s"resourceProfile should not have existed") + val rprofBuilder = new ResourceProfileBuilder() + val ereqs = new ExecutorResourceRequests() + ereqs.cores(checkId).memory("4g").memoryOverhead("2000m") + val treqs = new TaskResourceRequests() + treqs.cpus(checkId) + rprofBuilder.require(ereqs).require(treqs) + val rpShouldMatch = rprofBuilder.build + val equivProf = rpmanager.getEquivalentProfile(rpShouldMatch) + assert(equivProf.nonEmpty) + assert(equivProf.get.id == rpAlreadyExist.get.id, s"resourceProfile should have existed") + } } diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index b2f2c3632e454..29d3ef130ce57 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -193,6 +193,26 @@ class ResourceProfileSuite extends SparkFunSuite { assert(rprof.taskResources("cpus").amount === 1, "Task resources should have cpu") } + test("test ResourceProfiles equal") { + val rprofBuilder = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val eReq = new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprofBuilder.require(taskReq).require(eReq) + val rprof = rprofBuilder.build + + val rprofBuilder2 = new ResourceProfileBuilder() + val taskReq2 = new TaskResourceRequests().resource("gpu", 1) + val eReq2 = new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprofBuilder2.require(taskReq2).require(eReq2) + val rprof2 = rprofBuilder.build + rprof2.setResourceProfileId(rprof.id) + + assert(rprof === rprof2, "resource profile equality not working") + rprof2.setResourceProfileId(rprof.id + 1) + assert(rprof.id != rprof2.id, "resource profiles should not have same id") + assert(rprof.resourcesEqual(rprof2), "resource profile resourcesEqual not working") + } + test("Test ExecutorResourceRequests memory helpers") { val rprof = new ResourceProfileBuilder() val ereqs = new ExecutorResourceRequests() diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index 33594c0a50d14..0dd8be72dc904 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -65,7 +65,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { Thread.sleep(Random.nextInt(1000)) // Pass partitionId message in val message: String = context.partitionId().toString - val messages: ArrayBuffer[String] = context.allGather(message) + val messages: Array[String] = context.allGather(message) messages.toList.iterator } // Take a sorted list of all the partitionId messages diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 7666c6c7810cc..d648293fdbe06 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -72,7 +72,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } - assert(sc.maxNumConcurrentTasks() == 12) + assert(sc.maxNumConcurrentTasks(ResourceProfile.getOrCreateDefaultProfile(conf)) == 12) } test("compute max number of concurrent tasks can be launched when spark.task.cpus > 1") { @@ -86,7 +86,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(sc.getExecutorIds().length == 4) } // Each executor can only launch one task since `spark.task.cpus` is 2. - assert(sc.maxNumConcurrentTasks() == 4) + assert(sc.maxNumConcurrentTasks(ResourceProfile.getOrCreateDefaultProfile(conf)) == 4) } test("compute max number of concurrent tasks can be launched when some executors are busy") { @@ -126,7 +126,8 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(taskStarted.get()) assert(taskEnded.get() == false) // Assert we count in slots on both busy and free executors. - assert(sc.maxNumConcurrentTasks() == 4) + assert( + sc.maxNumConcurrentTasks(ResourceProfile.getOrCreateDefaultProfile(conf)) == 4) } } finally { sc.removeSparkListener(listener) @@ -187,8 +188,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo } test("extra resources from executor") { + import TestUtils._ + + val execCores = 3 val conf = new SparkConf() - .set(EXECUTOR_CORES, 1) + .set(EXECUTOR_CORES, execCores) .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test .set(EXECUTOR_INSTANCES, 0) // avoid errors about duplicate executor registrations .setMaster( @@ -244,7 +248,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], new Properties(), taskResources, bytebuffer))) val ts = backend.getTaskSchedulerImpl() - when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenReturn(taskDescs) + when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(taskDescs) backend.driverEndpoint.send(ReviveOffers) @@ -254,6 +258,9 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(execResources(GPU).assignedAddrs === Array("0")) } + // To avoid allocating any resources immediately after releasing the resource from the task to + // make sure that `availableAddrs` below won't change + when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(Seq.empty) backend.driverEndpoint.send( StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskResources)) @@ -294,7 +301,6 @@ private class CSMockExternalClusterManager extends ExternalClusterManager { when(ts.applicationAttemptId()).thenReturn(Some("attempt1")) when(ts.schedulingMode).thenReturn(SchedulingMode.FIFO) when(ts.nodeBlacklist()).thenReturn(Set.empty[String]) - when(ts.resourcesReqsPerTask).thenReturn(Seq.empty) ts } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 448638975d388..4c6033edff247 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -34,6 +34,8 @@ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.rdd.{DeterministicLevel, RDD} +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder, TaskResourceRequests} +import org.apache.spark.resource.ResourceUtils.{FPGA, GPU} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} @@ -2547,9 +2549,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi /** * Checks the DAGScheduler's internal logic for traversing an RDD DAG by making sure that - * getShuffleDependencies correctly returns the direct shuffle dependencies of a particular - * RDD. The test creates the following RDD graph (where n denotes a narrow dependency and s - * denotes a shuffle dependency): + * getShuffleDependenciesAndResourceProfiles correctly returns the direct shuffle dependencies + * of a particular RDD. The test creates the following RDD graph (where n denotes a narrow + * dependency and s denotes a shuffle dependency): * * A <------------s---------, * \ @@ -2558,7 +2560,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi * Here, the direct shuffle dependency of C is just the shuffle dependency on B. The direct * shuffle dependencies of E are the shuffle dependency on A and the shuffle dependency on C. */ - test("getShuffleDependencies correctly returns only direct shuffle parents") { + test("getShuffleDependenciesAndResourceProfiles correctly returns only direct shuffle parents") { val rddA = new MyRDD(sc, 2, Nil) val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(1)) val rddB = new MyRDD(sc, 2, Nil) @@ -2569,11 +2571,16 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val narrowDepD = new OneToOneDependency(rddD) val rddE = new MyRDD(sc, 1, List(shuffleDepA, narrowDepD), tracker = mapOutputTracker) - assert(scheduler.getShuffleDependencies(rddA) === Set()) - assert(scheduler.getShuffleDependencies(rddB) === Set()) - assert(scheduler.getShuffleDependencies(rddC) === Set(shuffleDepB)) - assert(scheduler.getShuffleDependencies(rddD) === Set(shuffleDepC)) - assert(scheduler.getShuffleDependencies(rddE) === Set(shuffleDepA, shuffleDepC)) + val (shuffleDepsA, _) = scheduler.getShuffleDependenciesAndResourceProfiles(rddA) + assert(shuffleDepsA === Set()) + val (shuffleDepsB, _) = scheduler.getShuffleDependenciesAndResourceProfiles(rddB) + assert(shuffleDepsB === Set()) + val (shuffleDepsC, _) = scheduler.getShuffleDependenciesAndResourceProfiles(rddC) + assert(shuffleDepsC === Set(shuffleDepB)) + val (shuffleDepsD, _) = scheduler.getShuffleDependenciesAndResourceProfiles(rddD) + assert(shuffleDepsD === Set(shuffleDepC)) + val (shuffleDepsE, _) = scheduler.getShuffleDependenciesAndResourceProfiles(rddE) + assert(shuffleDepsE === Set(shuffleDepA, shuffleDepC)) } test("SPARK-17644: After one stage is aborted for too many failed attempts, subsequent stages" + @@ -3141,6 +3148,234 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assertDataStructuresEmpty() } + test("test default resource profile") { + val rdd = sc.parallelize(1 to 10).map(x => (x, x)) + val (shuffledeps, resourceprofiles) = scheduler.getShuffleDependenciesAndResourceProfiles(rdd) + val rp = scheduler.mergeResourceProfilesForStage(resourceprofiles) + assert(rp.id == scheduler.sc.resourceProfileManager.defaultResourceProfile.id) + } + + test("test 1 resource profile") { + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build + + val rdd = sc.parallelize(1 to 10).map(x => (x, x)).withResources(rp1) + val (shuffledeps, resourceprofiles) = scheduler.getShuffleDependenciesAndResourceProfiles(rdd) + val rpMerged = scheduler.mergeResourceProfilesForStage(resourceprofiles) + val expectedid = Option(rdd.getResourceProfile).map(_.id) + assert(expectedid.isDefined) + assert(expectedid.get != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + assert(rpMerged.id == expectedid.get) + } + + test("test 2 resource profiles errors by default") { + import org.apache.spark.resource._ + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build + + val ereqs2 = new ExecutorResourceRequests().cores(2) + val treqs2 = new TaskResourceRequests().cpus(2) + val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build + + val rdd = sc.parallelize(1 to 10).withResources(rp1).map(x => (x, x)).withResources(rp2) + val error = intercept[IllegalArgumentException] { + val (shuffledeps, resourceprofiles) = scheduler.getShuffleDependenciesAndResourceProfiles(rdd) + scheduler.mergeResourceProfilesForStage(resourceprofiles) + }.getMessage() + + assert(error.contains("Multiple ResourceProfiles specified in the RDDs")) + } + + test("test 2 resource profile with merge conflict config true") { + afterEach() + val conf = new SparkConf() + conf.set(config.RESOURCE_PROFILE_MERGE_CONFLICTS.key, "true") + init(conf) + + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build + + val ereqs2 = new ExecutorResourceRequests().cores(2) + val treqs2 = new TaskResourceRequests().cpus(2) + val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build + + val rdd = sc.parallelize(1 to 10).withResources(rp1).map(x => (x, x)).withResources(rp2) + val (shuffledeps, resourceprofiles) = scheduler.getShuffleDependenciesAndResourceProfiles(rdd) + val mergedRp = scheduler.mergeResourceProfilesForStage(resourceprofiles) + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 4) + } + + test("test multiple resource profiles created from merging use same rp") { + afterEach() + val conf = new SparkConf() + conf.set(config.RESOURCE_PROFILE_MERGE_CONFLICTS.key, "true") + init(conf) + + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build + + val ereqs2 = new ExecutorResourceRequests().cores(2) + val treqs2 = new TaskResourceRequests().cpus(2) + val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build + + val rdd = sc.parallelize(1 to 10).withResources(rp1).map(x => (x, x)).withResources(rp2) + val (_, resourceprofiles) = scheduler.getShuffleDependenciesAndResourceProfiles(rdd) + val mergedRp = scheduler.mergeResourceProfilesForStage(resourceprofiles) + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 4) + + // test that instead of creating a new merged profile, we use the already created one + val rdd2 = sc.parallelize(1 to 10).withResources(rp1).map(x => (x, x)).withResources(rp2) + val (_, resourceprofiles2) = scheduler.getShuffleDependenciesAndResourceProfiles(rdd2) + val mergedRp2 = scheduler.mergeResourceProfilesForStage(resourceprofiles2) + assert(mergedRp2.id === mergedRp.id) + assert(mergedRp2.getTaskCpus.get == 2) + assert(mergedRp2.getExecutorCores.get == 4) + } + + test("test merge 2 resource profiles multiple configs") { + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(2) + val rp1 = new ResourceProfile(ereqs.requests, treqs.requests) + val ereqs2 = new ExecutorResourceRequests().cores(2) + val treqs2 = new TaskResourceRequests().cpus(1) + val rp2 = new ResourceProfile(ereqs2.requests, treqs2.requests) + var mergedRp = scheduler.mergeResourceProfiles(rp1, rp2) + + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 4) + + val ereqs3 = new ExecutorResourceRequests().cores(1).resource(GPU, 1, "disc") + val treqs3 = new TaskResourceRequests().cpus(1).resource(GPU, 1) + val rp3 = new ResourceProfile(ereqs3.requests, treqs3.requests) + val ereqs4 = new ExecutorResourceRequests().cores(2) + val treqs4 = new TaskResourceRequests().cpus(2) + val rp4 = new ResourceProfile(ereqs4.requests, treqs4.requests) + mergedRp = scheduler.mergeResourceProfiles(rp3, rp4) + + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 2) + assert(mergedRp.executorResources.size == 2) + assert(mergedRp.taskResources.size == 2) + assert(mergedRp.executorResources.get(GPU).get.amount == 1) + assert(mergedRp.executorResources.get(GPU).get.discoveryScript == "disc") + assert(mergedRp.taskResources.get(GPU).get.amount == 1) + + val ereqs5 = new ExecutorResourceRequests().cores(1).memory("3g") + .memoryOverhead("1g").pysparkMemory("2g").resource(GPU, 1, "disc") + val treqs5 = new TaskResourceRequests().cpus(1).resource(GPU, 1) + val rp5 = new ResourceProfile(ereqs5.requests, treqs5.requests) + val ereqs6 = new ExecutorResourceRequests().cores(8).resource(FPGA, 2, "fdisc") + val treqs6 = new TaskResourceRequests().cpus(2).resource(FPGA, 1) + val rp6 = new ResourceProfile(ereqs6.requests, treqs6.requests) + mergedRp = scheduler.mergeResourceProfiles(rp5, rp6) + + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 8) + assert(mergedRp.executorResources.size == 6) + assert(mergedRp.taskResources.size == 3) + assert(mergedRp.executorResources.get(GPU).get.amount == 1) + assert(mergedRp.executorResources.get(GPU).get.discoveryScript == "disc") + assert(mergedRp.taskResources.get(GPU).get.amount == 1) + assert(mergedRp.executorResources.get(FPGA).get.amount == 2) + assert(mergedRp.executorResources.get(FPGA).get.discoveryScript == "fdisc") + assert(mergedRp.taskResources.get(FPGA).get.amount == 1) + assert(mergedRp.executorResources.get(ResourceProfile.MEMORY).get.amount == 3072) + assert(mergedRp.executorResources.get(ResourceProfile.PYSPARK_MEM).get.amount == 2048) + assert(mergedRp.executorResources.get(ResourceProfile.OVERHEAD_MEM).get.amount == 1024) + + val ereqs7 = new ExecutorResourceRequests().cores(1).memory("3g") + .resource(GPU, 4, "disc") + val treqs7 = new TaskResourceRequests().cpus(1).resource(GPU, 1) + val rp7 = new ResourceProfile(ereqs7.requests, treqs7.requests) + val ereqs8 = new ExecutorResourceRequests().cores(1).resource(GPU, 2, "fdisc") + val treqs8 = new TaskResourceRequests().cpus(1).resource(GPU, 2) + val rp8 = new ResourceProfile(ereqs8.requests, treqs8.requests) + mergedRp = scheduler.mergeResourceProfiles(rp7, rp8) + + assert(mergedRp.getTaskCpus.get == 1) + assert(mergedRp.getExecutorCores.get == 1) + assert(mergedRp.executorResources.get(GPU).get.amount == 4) + assert(mergedRp.executorResources.get(GPU).get.discoveryScript == "disc") + assert(mergedRp.taskResources.get(GPU).get.amount == 2) + } + + test("test merge 3 resource profiles") { + afterEach() + val conf = new SparkConf() + conf.set(config.RESOURCE_PROFILE_MERGE_CONFLICTS.key, "true") + init(conf) + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfile(ereqs.requests, treqs.requests) + val ereqs2 = new ExecutorResourceRequests().cores(2) + val treqs2 = new TaskResourceRequests().cpus(1) + val rp2 = new ResourceProfile(ereqs2.requests, treqs2.requests) + val ereqs3 = new ExecutorResourceRequests().cores(3) + val treqs3 = new TaskResourceRequests().cpus(2) + val rp3 = new ResourceProfile(ereqs3.requests, treqs3.requests) + var mergedRp = scheduler.mergeResourceProfilesForStage(HashSet(rp1, rp2, rp3)) + + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 4) + } + + /** + * Checks the DAGScheduler's internal logic for traversing an RDD DAG by making sure that + * getShuffleDependenciesAndResourceProfiles correctly returns the direct shuffle dependencies + * of a particular RDD. The test creates the following RDD graph (where n denotes a narrow + * dependency and s denotes a shuffle dependency): + * + * A <------------s---------, + * \ + * B <--s-- C <--s-- D <--n------ E + * + * Here, the direct shuffle dependency of C is just the shuffle dependency on B. The direct + * shuffle dependencies of E are the shuffle dependency on A and the shuffle dependency on C. + */ + test("getShuffleDependenciesAndResourceProfiles returns deps and profiles correctly") { + import org.apache.spark.resource._ + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build + val ereqs2 = new ExecutorResourceRequests().cores(6) + val treqs2 = new TaskResourceRequests().cpus(2) + val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build + + val rddWithRp = new MyRDD(sc, 2, Nil).withResources(rp1) + val rddA = new MyRDD(sc, 2, Nil).withResources(rp1) + val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(1)) + val rddB = new MyRDD(sc, 2, Nil) + val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(1)) + val rddWithRpDep = new OneToOneDependency(rddWithRp) + val rddC = new MyRDD(sc, 1, List(rddWithRpDep, shuffleDepB)).withResources(rp2) + val shuffleDepC = new ShuffleDependency(rddC, new HashPartitioner(1)) + val rddD = new MyRDD(sc, 1, List(shuffleDepC)) + val narrowDepD = new OneToOneDependency(rddD) + val rddE = new MyRDD(sc, 1, List(shuffleDepA, narrowDepD), tracker = mapOutputTracker) + + val (shuffleDepsA, rprofsA) = scheduler.getShuffleDependenciesAndResourceProfiles(rddA) + assert(shuffleDepsA === Set()) + assert(rprofsA === Set(rp1)) + val (shuffleDepsB, rprofsB) = scheduler.getShuffleDependenciesAndResourceProfiles(rddB) + assert(shuffleDepsB === Set()) + assert(rprofsB === Set()) + val (shuffleDepsC, rprofsC) = scheduler.getShuffleDependenciesAndResourceProfiles(rddC) + assert(shuffleDepsC === Set(shuffleDepB)) + assert(rprofsC === Set(rp1, rp2)) + val (shuffleDepsD, rprofsD) = scheduler.getShuffleDependenciesAndResourceProfiles(rddD) + assert(shuffleDepsD === Set(shuffleDepC)) + assert(rprofsD === Set()) + val (shuffleDepsE, rprofsE) = scheduler.getShuffleDependenciesAndResourceProfiles(rddE) + assert(shuffleDepsE === Set(shuffleDepA, shuffleDepC)) + assert(rprofsE === Set()) + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 9f593e0039adc..7ead51bc691fb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.Map import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -71,7 +72,7 @@ private class DummySchedulerBackend extends SchedulerBackend { def stop(): Unit = {} def reviveOffers(): Unit = {} def defaultParallelism(): Int = 1 - def maxNumConcurrentTasks(): Int = 0 + def maxNumConcurrentTasks(rp: ResourceProfile): Int = 0 } private class DummyTaskScheduler extends TaskScheduler { diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index 8cb6268f85d36..9ec088aaddddd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -21,6 +21,7 @@ import java.util.Properties import org.apache.spark.{Partition, SparkEnv, TaskContext} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.resource.ResourceProfile class FakeTask( stageId: Int, @@ -42,7 +43,12 @@ object FakeTask { * locations for each task (given as varargs) if this sequence is not empty. */ def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, prefLocs: _*) + createTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, prefLocs: _*) + } + + def createTaskSet(numTasks: Int, rpId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { + createTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, rpId, prefLocs: _*) } def createTaskSet( @@ -50,7 +56,8 @@ object FakeTask { stageId: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, stageId, stageAttemptId, priority = 0, prefLocs: _*) + createTaskSet(numTasks, stageId, stageAttemptId, priority = 0, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, prefLocs: _*) } def createTaskSet( @@ -58,6 +65,7 @@ object FakeTask { stageId: Int, stageAttemptId: Int, priority: Int, + rpId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") @@ -65,7 +73,7 @@ object FakeTask { val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil) } - new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null, rpId) } def createShuffleMapTaskSet( @@ -91,11 +99,21 @@ object FakeTask { }, prefLocs(i), new Properties, SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array()) } - new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } def createBarrierTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createBarrierTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, prefLocs: _*) + createBarrierTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, + rpId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, prefLocs: _*) + } + + def createBarrierTaskSet( + numTasks: Int, + rpId: Int, + prefLocs: Seq[TaskLocation]*): TaskSet = { + createBarrierTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, + rpId = rpId, prefLocs: _*) } def createBarrierTaskSet( @@ -103,6 +121,7 @@ object FakeTask { stageId: Int, stageAttemptId: Int, priority: Int, + rpId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") @@ -110,6 +129,6 @@ object FakeTask { val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil, isBarrier = true) } - new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null, rpId) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala index b953add9d58cb..d9de976c789d4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala @@ -22,6 +22,7 @@ import java.util.Properties import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.internal.config.SCHEDULER_ALLOCATION_FILE +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.SchedulingMode._ /** @@ -39,7 +40,8 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(stageId, i, Nil) } - new TaskSetManager(taskScheduler, new TaskSet(tasks, stageId, 0, 0, null), 0) + new TaskSetManager(taskScheduler, new TaskSet(tasks, stageId, 0, 0, null, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), 0) } def scheduleTaskAndVerifyId(taskId: Int, rootPool: Pool, expectedStageId: Int): Unit = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index dff8975a4fe49..0874163b0e946 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -36,6 +36,7 @@ import org.apache.spark.TaskState._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.SCHEDULER_REVIVE_INTERVAL import org.apache.spark.rdd.RDD +import org.apache.spark.resource.ResourceProfile import org.apache.spark.util.{CallSite, ThreadUtils, Utils} /** @@ -385,7 +386,7 @@ private[spark] abstract class MockBackend( }.toIndexedSeq } - override def maxNumConcurrentTasks(): Int = 0 + override def maxNumConcurrentTasks(rp: ResourceProfile): Int = 0 /** * This is called by the scheduler whenever it has tasks it would like to schedule, when a tasks @@ -406,9 +407,9 @@ private[spark] abstract class MockBackend( (taskDescription, task) } newTasks.foreach { case (taskDescription, _) => + freeCores -= taskScheduler.CPUS_PER_TASK executorIdToExecutor(taskDescription.executorId).freeCores -= taskScheduler.CPUS_PER_TASK } - freeCores -= newTasks.size * taskScheduler.CPUS_PER_TASK assignedTasksWaitingToRun ++= newTasks } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index e7ecf847ff4f4..056c34278c1ea 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -31,6 +31,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, TaskResourceRequests} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.util.ManualClock @@ -40,7 +41,7 @@ class FakeSchedulerBackend extends SchedulerBackend { def stop(): Unit = {} def reviveOffers(): Unit = {} def defaultParallelism(): Int = 1 - def maxNumConcurrentTasks(): Int = 0 + def maxNumConcurrentTasks(rp: ResourceProfile): Int = 0 } class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach @@ -195,6 +196,240 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!failedTaskSet) } + private def setupTaskSchedulerForLocalityTests(clock: ManualClock): TaskSchedulerImpl = { + val conf = new SparkConf() + sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) + val taskScheduler = new TaskSchedulerImpl(sc, + sc.conf.get(config.TASK_MAX_FAILURES), + clock = clock) { + override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { + new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt, clock) + } + override def shuffleOffers(offers: IndexedSeq[WorkerOffer]): IndexedSeq[WorkerOffer] = { + // Don't shuffle the offers around for this test. Instead, we'll just pass in all + // the permutations we care about directly. + offers + } + } + // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. + new DAGScheduler(sc, taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} + + override def executorAdded(execId: String, host: String): Unit = {} + } + taskScheduler.initialize(new FakeSchedulerBackend) + val taskSet = FakeTask.createTaskSet(8, 1, 1, + Seq(TaskLocation("host1", "exec1")), + Seq(TaskLocation("host1", "exec1")), + Seq(TaskLocation("host1", "exec1")), + Seq(TaskLocation("host1", "exec1")), + Seq(TaskLocation("host1", "exec1")), + Seq(TaskLocation("host1", "exec1")), + Seq(TaskLocation("host1", "exec1")), + Seq(TaskLocation("host1", "exec1")) + ) + + // Offer resources first so that when the taskset is submitted it can initialize + // with proper locality level. Otherwise, ANY would be the only locality level. + // See TaskSetManager.computeValidLocalityLevels() + // This begins the task set as PROCESS_LOCAL locality level + taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("exec1", "host1", 1))) + taskScheduler.submitTasks(taskSet) + taskScheduler + } + + test("SPARK-18886 - partial offers (isAllFreeResources = false) reset timer before " + + "any resources have been rejected") { + val clock = new ManualClock() + // All tasks created here are local to exec1, host1. + // Locality level starts at PROCESS_LOCAL. + val taskScheduler = setupTaskSchedulerForLocalityTests(clock) + // Locality levels increase at 3000 ms. + val advanceAmount = 3000 + + // Advancing clock increases locality level to NODE_LOCAL. + clock.advance(advanceAmount) + + // If there hasn't yet been any full resource offers, + // partial resource (isAllFreeResources = false) offers reset delay scheduling + // if this and previous offers were accepted. + // This line resets the timer and locality level is reset to PROCESS_LOCAL. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec1", "host1", 1)), + isAllFreeResources = false) + .flatten.length === 1) + + // This NODE_LOCAL task should not be accepted. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec2", "host1", 1)), + isAllFreeResources = false) + .flatten.isEmpty) + } + + test("SPARK-18886 - delay scheduling timer is reset when it accepts all resources offered when " + + "isAllFreeResources = true") { + val clock = new ManualClock() + // All tasks created here are local to exec1, host1. + // Locality level starts at PROCESS_LOCAL. + val taskScheduler = setupTaskSchedulerForLocalityTests(clock) + // Locality levels increase at 3000 ms. + val advanceAmount = 3000 + + // Advancing clock increases locality level to NODE_LOCAL. + clock.advance(advanceAmount) + + // If there are no rejects on an all resource offer, delay scheduling is reset. + // This line resets the timer and locality level is reset to PROCESS_LOCAL. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec1", "host1", 1)), + isAllFreeResources = true) + .flatten.length === 1) + + // This NODE_LOCAL task should not be accepted. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec2", "host1", 1)), + isAllFreeResources = false) + .flatten.isEmpty) + } + + test("SPARK-18886 - partial resource offers (isAllFreeResources = false) reset " + + "time if last full resource offer (isAllResources = true) was accepted as well as any " + + "following partial resource offers") { + val clock = new ManualClock() + // All tasks created here are local to exec1, host1. + // Locality level starts at PROCESS_LOCAL. + val taskScheduler = setupTaskSchedulerForLocalityTests(clock) + // Locality levels increase at 3000 ms. + val advanceAmount = 3000 + + // PROCESS_LOCAL full resource offer is accepted. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec1", "host1", 1)), + isAllFreeResources = true) + .flatten.length === 1) + + // Advancing clock increases locality level to NODE_LOCAL. + clock.advance(advanceAmount) + + // PROCESS_LOCAL partial resource is accepted. + // Since all offers have been accepted since the last full resource offer + // (this one and the previous one), delay scheduling is reset. + // This line resets the timer and locality level is reset to PROCESS_LOCAL. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec1", "host1", 1)), + isAllFreeResources = false) + .flatten.length === 1) + + // Advancing clock increases locality level to NODE_LOCAL + clock.advance(advanceAmount) + + // PROCESS_LOCAL partial resource is accepted + // Since all offers have been accepted since the last full resource offer + // (one previous full offer, one previous partial offer, and this partial offer), + // delay scheduling is reset. + // This line resets the timer and locality level is reset to PROCESS_LOCAL. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec1", "host1", 1)), + isAllFreeResources = false) + .flatten.length === 1) + + // This NODE_LOCAL task should not be accepted. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec2", "host1", 1)), + isAllFreeResources = false) + .flatten.isEmpty) + } + + // This tests two cases + // 1. partial resource offer doesn't reset timer after full resource offer had rejected resources + // 2. partial resource offer doesn't reset timer after partial resource offer + // had rejected resources + test("SPARK-18886 - partial resource offers (isAllFreeResources = false) do not reset " + + "time if any offer was rejected since last full offer was fully accepted") { + val clock = new ManualClock() + // All tasks created here are local to exec1, host1. + // Locality level starts at PROCESS_LOCAL. + val taskScheduler = setupTaskSchedulerForLocalityTests(clock) + // Locality levels increase at 3000 ms. + val advanceAmount = 3000 + + // case 1 from test description above. + // NODE_LOCAL full resource offer is rejected, so delay scheduling is not reset. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec2", "host1", 1)), + isAllFreeResources = true) + .flatten.isEmpty) + + // Advancing clock increases locality level to NODE_LOCAL + clock.advance(advanceAmount) + + // PROCESS_LOCAL partial resource is accepted, + // but because preceding full resource offer was rejected, delay scheduling is not reset. + // Locality level remains at NODE_LOCAL. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec1", "host1", 1)), + isAllFreeResources = false) + .flatten.length === 1) + + // Even though we launched a local task above, we still utilize non-local exec2. + // This is the behavior change to fix SPARK-18886. + // Locality level remains NODE_LOCAL after this clock advance. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec2", "host1", 1)), + isAllFreeResources = false) + .flatten.length === 1) + + + // case 2 from test description above. + // PROCESS_LOCAL full resource offer is accepted, resetting delay scheduling. + // This line resets the timer and locality level is reset to PROCESS_LOCAL. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec1", "host1", 1)), + isAllFreeResources = true) + .flatten.length === 1) + + // Partial resource offer: NODE_LOCAL exec 2 is rejected, PROCESS_LOCAL exec1 is accepted. + // Since there were rejects, delay scheduling is not reset, and follow up partial offers + // will not reset delay scheduling, even if they are accepted. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec2", "host1", 1), WorkerOffer("exec1", "host1", 1)), + isAllFreeResources = false) + .flatten.size === 1) + + // Advancing clock increases locality level to NODE_LOCAL + clock.advance(advanceAmount) + + // PROCESS_LOCAL partial resource is accepted, but does not reset delay scheduling + // as described above. + // Locality level remains at NODE_LOCAL. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec1", "host1", 1)), + isAllFreeResources = false) + .flatten.length === 1) + + // NODE_LOCAL partial resource offer is accepted, + // verifying locality level was not reset to PROCESS_LOCAL by above offer. + assert(taskScheduler + .resourceOffers( + IndexedSeq(WorkerOffer("exec2", "host1", 1)), + isAllFreeResources = false) + .flatten.length === 1) + } + test("Scheduler does not crash when tasks are not serializable") { val taskCpus = 2 val taskScheduler = setupSchedulerWithMaster( @@ -202,7 +437,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B config.CPUS_PER_TASK.key -> taskCpus.toString) val numFreeCores = 1 val taskSet = new TaskSet( - Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), + 0, 0, 0, null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val multiCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", taskCpus), new WorkerOffer("executor1", "host1", numFreeCores)) taskScheduler.submitTasks(taskSet) @@ -216,7 +452,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // still be processed without error taskScheduler.submitTasks(FakeTask.createTaskSet(1)) val taskSet2 = new TaskSet( - Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 1, 0, 0, null) + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), + 1, 0, 0, null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) taskScheduler.submitTasks(taskSet2) taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten assert(taskDescriptions.map(_.executorId) === Seq("executor0")) @@ -758,7 +995,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // that are explicitly blacklisted, plus those that have *any* executors blacklisted. val nodesForBlacklistedExecutors = offers.filter { offer => execBlacklist.contains(offer.executorId) - }.map(_.host).toSet.toSeq + }.map(_.host).distinct val nodesWithAnyBlacklisting = (nodeBlacklist ++ nodesForBlacklistedExecutors).toSet // Similarly, figure out which executors have any blacklisting. This means all executors // that are explicitly blacklisted, plus all executors on nodes that are blacklisted. @@ -898,18 +1135,17 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } // Here is the main check of this test -- we have the same offers again, and we schedule it - // successfully. Because the scheduler first tries to schedule with locality in mind, at first - // it won't schedule anything on executor1. But despite that, we don't abort the job. Then the - // scheduler tries for ANY locality, and successfully schedules tasks on executor1. + // successfully. Because the scheduler tries to schedule with locality in mind, at first + // it won't schedule anything on executor1. But despite that, we don't abort the job. val secondTaskAttempts = taskScheduler.resourceOffers(offers).flatten - assert(secondTaskAttempts.size == 2) - secondTaskAttempts.foreach { taskAttempt => assert("executor1" === taskAttempt.executorId) } + assert(secondTaskAttempts.isEmpty) assert(!failedTaskSet) } test("SPARK-16106 locality levels updated if executor added to existing host") { val taskScheduler = setupScheduler() + taskScheduler.resourceOffers(IndexedSeq(new WorkerOffer("executor0", "host0", 1))) taskScheduler.submitTasks(FakeTask.createTaskSet(2, stageId = 0, stageAttemptId = 0, (0 until 2).map { _ => Seq(TaskLocation("host0", "executor2")) }: _* )) @@ -1135,6 +1371,96 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(0 === taskDescriptions.length) } + test("don't schedule for a barrier taskSet if available slots are less than " + + "pending tasks gpus limiting") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString, + "spark.executor.resource.gpu.amount" -> "1", "spark.task.resource.gpu.amount" -> "1") + + val numFreeCores = 3 + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0").toBuffer)), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0").toBuffer))) + val attempt1 = FakeTask.createBarrierTaskSet(3) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(0 === taskDescriptions.length) + } + + test("schedule tasks for a barrier taskSet if all tasks can be launched together gpus") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString, + "spark.executor.resource.gpu.amount" -> "1", "spark.task.resource.gpu.amount" -> "1") + + val numFreeCores = 3 + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0").toBuffer)), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0").toBuffer)), + new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49629"), + Map("gpu" -> Seq("0").toBuffer))) + val attempt1 = FakeTask.createBarrierTaskSet(3) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(3 === taskDescriptions.length) + } + + // barrier scheduling doesn't yet work with dynamic allocation but test it with another + // ResourceProfile anyway to make sure code path works when it is supported + test("schedule tasks for a barrier taskSet if all tasks can be launched together " + + "diff ResourceProfile") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString) + val execReqs = new ExecutorResourceRequests().cores(2).resource("gpu", 2) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + val rp = new ResourceProfile(execReqs.requests, taskReqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val numFreeCores = 2 + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0", "1").toBuffer), rp.id), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0", "1").toBuffer), rp.id)) + val attempt1 = FakeTask.createBarrierTaskSet(3, rpId = rp.id) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(3 === taskDescriptions.length) + } + + test("schedule tasks for a barrier taskSet if all tasks can be launched together " + + "diff ResourceProfile, but not enough gpus") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString) + val execReqs = new ExecutorResourceRequests().cores(2).resource("gpu", 2) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + val rp = new ResourceProfile(execReqs.requests, taskReqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val numFreeCores = 2 + // make each of the worker offers only have 1 GPU, thus making it not enough + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0").toBuffer), rp.id), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0").toBuffer), rp.id)) + val attempt1 = FakeTask.createBarrierTaskSet(3, rpId = rp.id) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(0 === taskDescriptions.length) + } + test("schedule tasks for a barrier taskSet if all tasks can be launched together") { val taskCpus = 2 val taskScheduler = setupSchedulerWithMaster( @@ -1165,8 +1491,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625")), new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627")), new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49629"))) - val barrier = FakeTask.createBarrierTaskSet(3, stageId = 0, stageAttemptId = 0, priority = 1) - val highPrio = FakeTask.createTaskSet(1, stageId = 1, stageAttemptId = 0, priority = 0) + val barrier = FakeTask.createBarrierTaskSet(3, stageId = 0, stageAttemptId = 0, priority = 1, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val highPrio = FakeTask.createTaskSet(1, stageId = 1, stageAttemptId = 0, priority = 0, + rpId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // submit highPrio and barrier taskSet taskScheduler.submitTasks(highPrio) @@ -1289,6 +1617,93 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(ArrayBuffer("1") === taskDescriptions(1).resources.get(GPU).get.addresses) } + test("Scheduler correctly accounts for GPUs per task with fractional amount") { + val taskCpus = 1 + val taskGpus = 0.33 + val executorGpus = 1 + val executorCpus = 4 + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> taskGpus.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString) + val taskSet = FakeTask.createTaskSet(5) + + val numFreeCores = 4 + val resources = Map(GPU -> ArrayBuffer("0", "0", "0")) + val singleCoreWorkerOffers = + IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, resources)) + + taskScheduler.submitTasks(taskSet) + // Launch tasks on executor that satisfies resource requirements. + var taskDescriptions = taskScheduler.resourceOffers(singleCoreWorkerOffers).flatten + assert(3 === taskDescriptions.length) + assert(!failedTaskSet) + assert(ArrayBuffer("0") === taskDescriptions(0).resources.get(GPU).get.addresses) + assert(ArrayBuffer("0") === taskDescriptions(1).resources.get(GPU).get.addresses) + assert(ArrayBuffer("0") === taskDescriptions(2).resources.get(GPU).get.addresses) + } + + test("Scheduler works with multiple ResourceProfiles and gpus") { + val taskCpus = 1 + val taskGpus = 1 + val executorGpus = 4 + val executorCpus = 4 + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> taskGpus.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString) + + val ereqs = new ExecutorResourceRequests().cores(6).resource(GPU, 6) + val treqs = new TaskResourceRequests().cpus(2).resource(GPU, 2) + val rp = new ResourceProfile(ereqs.requests, treqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + val taskSet = FakeTask.createTaskSet(3) + val rpTaskSet = FakeTask.createTaskSet(5, stageId = 1, stageAttemptId = 0, + priority = 0, rpId = rp.id) + + val resourcesDefaultProf = Map(GPU -> ArrayBuffer("0", "1", "2", "3")) + val resources = Map(GPU -> ArrayBuffer("4", "5", "6", "7", "8", "9")) + + val workerOffers = + IndexedSeq(new WorkerOffer("executor0", "host0", 2, None, resourcesDefaultProf), + new WorkerOffer("executor1", "host1", 6, None, resources, rp.id)) + taskScheduler.submitTasks(taskSet) + taskScheduler.submitTasks(rpTaskSet) + // should have 2 for default profile and 2 for additional resource profile + var taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(5 === taskDescriptions.length) + var has2Gpus = 0 + var has1Gpu = 0 + for (tDesc <- taskDescriptions) { + assert(tDesc.resources.contains(GPU)) + if (tDesc.resources(GPU).addresses.size == 2) { + has2Gpus += 1 + } + if (tDesc.resources(GPU).addresses.size == 1) { + has1Gpu += 1 + } + } + assert(has2Gpus == 3) + assert(has1Gpu == 2) + + val resources3 = Map(GPU -> ArrayBuffer("14", "15", "16", "17", "18", "19")) + + // clear the first 2 worker offers so they don't have any room and add a third + // for the resource profile + val workerOffers3 = IndexedSeq( + new WorkerOffer("executor0", "host0", 0, None, Map.empty), + new WorkerOffer("executor1", "host1", 0, None, Map.empty, rp.id), + new WorkerOffer("executor2", "host2", 6, None, resources3, rp.id)) + taskDescriptions = taskScheduler.resourceOffers(workerOffers3).flatten + assert(2 === taskDescriptions.length) + assert(taskDescriptions.head.resources.contains(GPU)) + assert(2 == taskDescriptions.head.resources(GPU).addresses.size) + } + /** * Used by tests to simulate a task failure. This calls the failure handler explicitly, to ensure * that all the state is updated when this method returns. Otherwise, there's no way to know when 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 b740e357903a2..4978be3e04c1e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -34,6 +34,8 @@ import org.scalatest.concurrent.Eventually import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests.SKIP_VALIDATE_CORES_TESTING +import org.apache.spark.resource.{ResourceInformation, ResourceProfile} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -213,7 +215,6 @@ class TaskSetManagerSuite super.afterEach() } - test("TaskSet with no preferences") { sc = new SparkContext("local", "test") sched = new FakeTaskScheduler(sc, ("exec1", "host1")) @@ -224,7 +225,7 @@ class TaskSetManagerSuite // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOption.isDefined) clock.advance(1) @@ -245,7 +246,7 @@ class TaskSetManagerSuite // First three offers should all find tasks for (i <- 0 until 3) { - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") @@ -253,7 +254,7 @@ class TaskSetManagerSuite assert(sched.startedTasks.toSet === Set(0, 1, 2)) // Re-offer the host -- now we should get no more tasks - assert(manager.resourceOffer("exec1", "host1", NO_PREF) === None) + assert(manager.resourceOffer("exec1", "host1", NO_PREF)._1 === None) // Finish the first two tasks manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdatesByTask(0))) @@ -276,12 +277,12 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // An executor that is not NODE_LOCAL should be rejected. - assert(manager.resourceOffer("execC", "host2", ANY) === None) + assert(manager.resourceOffer("execC", "host2", ANY)._1 === None) // Because there are no alive PROCESS_LOCAL executors, the base locality level should be // NODE_LOCAL. So, we should schedule the task on this offered NODE_LOCAL executor before // any of the locality wait timers expire. - assert(manager.resourceOffer("execA", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY)._1.get.index === 0) } test("basic delay scheduling") { @@ -296,22 +297,22 @@ class TaskSetManagerSuite val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 0) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL)._1 === None) clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 3) should // get chosen before the noPref task - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL)._1.get.index == 2) // Offer host2, exec2, at NODE_LOCAL level: we should choose task 2 - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL).get.index == 1) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL)._1.get.index == 1) // Offer host2, exec2 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL)._1 === None) clock.advance(LOCALITY_WAIT_MS) - assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) + assert(manager.resourceOffer("exec2", "host2", NO_PREF)._1.get.index == 3) } test("we do not need to delay scheduling when we only have noPref tasks in the queue") { @@ -325,10 +326,10 @@ class TaskSetManagerSuite val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) - assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL).get.index === 1) - assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL) == None) - assert(manager.resourceOffer("exec3", "host2", NO_PREF).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL)._1.get.index === 0) + assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL)._1.get.index === 1) + assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL)._1 === None) + assert(manager.resourceOffer("exec3", "host2", NO_PREF)._1.get.index === 2) } test("delay scheduling with fallback") { @@ -342,33 +343,55 @@ class TaskSetManagerSuite Seq(TaskLocation("host3")), Seq(TaskLocation("host2")) ) + sc.conf.set(config.LEGACY_LOCALITY_WAIT_RESET, true) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 0) // Offer host1 again: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY)._1 === None) clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 1) // Offer host1 again: third task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 2) // Offer host2: fifth task (also on host2) should get chosen - assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 4) + assert(manager.resourceOffer("exec2", "host2", ANY)._1.get.index === 4) // Now that we've launched a local task, we should no longer launch the task for host3 - assert(manager.resourceOffer("exec2", "host2", ANY) === None) + assert(manager.resourceOffer("exec2", "host2", ANY)._1 === None) clock.advance(LOCALITY_WAIT_MS) + // offers not accepted due to task set zombies are not delay schedule rejects + manager.isZombie = true + val (taskDesciption, delayReject) = manager.resourceOffer("exec2", "host2", ANY) + assert(taskDesciption.isEmpty) + assert(delayReject === false) + manager.isZombie = false + + // offers not accepted due to blacklisting are not delay schedule rejects + val tsmSpy = spy(manager) + val blacklist = mock(classOf[TaskSetBlacklist]) + when(tsmSpy.taskSetBlacklistHelperOpt).thenReturn(Some(blacklist)) + when(blacklist.isNodeBlacklistedForTaskSet(any())).thenReturn(true) + val (blacklistTask, blackListReject) = tsmSpy.resourceOffer("exec2", "host2", ANY) + assert(blacklistTask.isEmpty) + assert(blackListReject === false) + // After another delay, we can go ahead and launch that task non-locally - assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) + assert(manager.resourceOffer("exec2", "host2", ANY)._1.get.index === 3) + + // offers not accepted due to no pending tasks are not delay schedule rejects + val (noPendingTask, noPendingReject) = manager.resourceOffer("exec2", "host2", ANY) + assert(noPendingTask.isEmpty) + assert(noPendingReject === false) } test("delay scheduling with failed hosts") { @@ -384,28 +407,28 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 0) // After this, nothing should get chosen, because we have separated tasks with unavailable // preference from the noPrefPendingTasks - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY)._1 === None) // Now mark host2 as dead sched.removeExecutor("exec2") manager.executorLost("exec2", "host2", SlaveLost()) // nothing should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY)._1 === None) clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 2) // all finished - assert(manager.resourceOffer("exec1", "host1", ANY) === None) - assert(manager.resourceOffer("exec2", "host2", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY)._1 === None) + assert(manager.resourceOffer("exec2", "host2", ANY)._1 === None) } test("task result lost") { @@ -416,14 +439,14 @@ class TaskSetManagerSuite clock.advance(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 0) // Tell it the task has finished but the result was lost. manager.handleFailedTask(0, TaskState.FINISHED, TaskResultLost) assert(sched.endedTasks(0) === TaskResultLost) // Re-offer the host -- now we should get task 0 again. - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 0) } test("repeated failures lead to task set abortion") { @@ -437,7 +460,7 @@ class TaskSetManagerSuite // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. (1 to manager.maxTaskFailures).foreach { index => - val offerResult = manager.resourceOffer("exec1", "host1", ANY) + val offerResult = manager.resourceOffer("exec1", "host1", ANY)._1 assert(offerResult.isDefined, "Expect resource offer on iteration %s to return a task".format(index)) assert(offerResult.get.index === 0) @@ -473,7 +496,7 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL)._1 assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -484,15 +507,15 @@ class TaskSetManagerSuite assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1 fails after failure 1 due to blacklist - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL)._1.isEmpty) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL)._1.isEmpty) + assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL)._1.isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.isEmpty) } // Run the task on exec1.1 - should work, and then fail it on exec1.1 { - val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL) + val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL)._1 assert(offerResult.isDefined, "Expect resource offer to return a task for exec1.1, offerResult = " + offerResult) @@ -504,12 +527,12 @@ class TaskSetManagerSuite assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1.1 fails after failure 2 due to blacklist - assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL)._1.isEmpty) } // Run the task on exec2 - should work, and then fail it on exec2 { - val offerResult = manager.resourceOffer("exec2", "host2", ANY) + val offerResult = manager.resourceOffer("exec2", "host2", ANY)._1 assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -520,7 +543,7 @@ class TaskSetManagerSuite assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec2 fails after failure 3 due to blacklist - assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) + assert(manager.resourceOffer("exec2", "host2", ANY)._1.isEmpty) } // Despite advancing beyond the time for expiring executors from within the blacklist, @@ -528,17 +551,17 @@ class TaskSetManagerSuite clock.advance(rescheduleDelay) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL)._1 assert(offerResult.isEmpty) } { - val offerResult = manager.resourceOffer("exec3", "host3", ANY) + val offerResult = manager.resourceOffer("exec3", "host3", ANY)._1 assert(offerResult.isDefined) assert(offerResult.get.index === 0) assert(offerResult.get.executorId === "exec3") - assert(manager.resourceOffer("exec3", "host3", ANY).isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY)._1.isEmpty) // Cause exec3 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) @@ -597,14 +620,14 @@ class TaskSetManagerSuite manager.executorAdded() sched.addExecutor("execC", "host2") manager.executorAdded() - assert(manager.resourceOffer("exec1", "host1", ANY).isDefined) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.isDefined) sched.removeExecutor("execA") manager.executorLost( "execA", "host1", ExecutorExited(143, false, "Terminated for reason unrelated to running tasks")) assert(!sched.taskSetsFailed.contains(taskSet.id)) - assert(manager.resourceOffer("execC", "host2", ANY).isDefined) + assert(manager.resourceOffer("execC", "host2", ANY)._1.isDefined) sched.removeExecutor("execC") manager.executorLost( "execC", "host2", ExecutorExited(1, true, "Terminated due to issue with running tasks")) @@ -632,12 +655,12 @@ class TaskSetManagerSuite clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL - assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) + assert(manager.resourceOffer("execC", "host3", RACK_LOCAL)._1 === None) // Task 0 can be scheduled with ANY - assert(manager.resourceOffer("execC", "host3", ANY).get.index === 0) + assert(manager.resourceOffer("execC", "host3", ANY)._1.get.index === 0) // Offer host2 // Task 1 can be scheduled with RACK_LOCAL - assert(manager.resourceOffer("execB", "host2", RACK_LOCAL).get.index === 1) + assert(manager.resourceOffer("execB", "host2", RACK_LOCAL)._1.get.index === 1) } test("do not emit warning when serialized task is small") { @@ -648,7 +671,7 @@ class TaskSetManagerSuite assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 0) assert(!manager.emittedTaskSizeWarning) } @@ -657,12 +680,13 @@ class TaskSetManagerSuite sc = new SparkContext("local", "test") sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = new TaskSet(Array(new LargeTask(0)), 0, 0, 0, null) + val taskSet = new TaskSet(Array(new LargeTask(0)), 0, 0, 0, + null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.get.index === 0) assert(manager.emittedTaskSizeWarning) } @@ -672,7 +696,8 @@ class TaskSetManagerSuite sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = new TaskSet( - Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), + 0, 0, 0, null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) intercept[TaskNotSerializableException] { @@ -743,18 +768,19 @@ class TaskSetManagerSuite val singleTask = new ShuffleMapTask(0, 0, null, new Partition { override def index: Int = 0 }, Seq(TaskLocation("host1", "execA")), new Properties, null) - val taskSet = new TaskSet(Array(singleTask), 0, 0, 0, null) + val taskSet = new TaskSet(Array(singleTask), 0, 0, 0, + null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) // Offer host1, which should be accepted as a PROCESS_LOCAL location // by the one task in the task set - val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL).get + val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL)._1.get // Mark the task as available for speculation, and then offer another resource, // which should be used to launch a speculative copy of the task. manager.speculatableTasks += singleTask.partitionId manager.addPendingTask(singleTask.partitionId, speculatable = true) - val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY).get + val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY)._1.get assert(manager.runningTasks === 2) assert(manager.isZombie === false) @@ -840,7 +866,7 @@ class TaskSetManagerSuite "exec1" -> "host1", "exec3" -> "host3", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + val taskOption = manager.resourceOffer(exec, host, NO_PREF)._1 assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -866,7 +892,7 @@ class TaskSetManagerSuite assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY) + val taskOption = manager.resourceOffer("exec2", "host2", ANY)._1 assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -895,20 +921,20 @@ class TaskSetManagerSuite val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL)._1.get.index === 0) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL)._1 === None) + assert(manager.resourceOffer("execA", "host1", NO_PREF)._1.get.index == 1) manager.speculatableTasks += 1 manager.addPendingTask(1, speculatable = true) clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task - assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) + assert(manager.resourceOffer("execA", "host1", NO_PREF)._1.get.index === 2) // schedule the speculative task - assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) + assert(manager.resourceOffer("execB", "host2", NO_PREF)._1.get.index === 1) clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks - assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) + assert(manager.resourceOffer("execB", "host2", ANY)._1.get.index === 3) } test("node-local tasks should be scheduled right away " + @@ -925,13 +951,13 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) - assert(manager.resourceOffer("execA", "host2", NODE_LOCAL).get.index === 1) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL).get.index === 3) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL) === None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL)._1.get.index === 0) + assert(manager.resourceOffer("execA", "host2", NODE_LOCAL)._1.get.index === 1) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL)._1.get.index === 3) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL)._1 === None) // schedule no-preference after node local ones - assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) + assert(manager.resourceOffer("execA", "host3", NO_PREF)._1.get.index === 2) } test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") @@ -947,13 +973,13 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 3) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL)._1.get.index === 2) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL)._1.get.index === 3) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 1) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL)._1.get.index === 0) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL)._1.get.index === 1) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL)._1 === None) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL)._1 === None) } test("SPARK-4939: no-pref tasks should be scheduled after process-local tasks finished") { @@ -967,13 +993,13 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1) - assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL).get.index === 2) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL)._1.get.index === 1) + assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL)._1.get.index === 2) // no-pref tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL) == None) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 0) - assert(manager.resourceOffer("execA", "host1", ANY) == None) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL)._1 === None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL)._1 === None) + assert(manager.resourceOffer("execA", "host1", NO_PREF)._1.get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY)._1 === None) } test("Ensure TaskSetManager is usable after addition of levels") { @@ -1053,11 +1079,11 @@ class TaskSetManagerSuite } // Offer resources for 4 tasks to start for ((k, v) <- List( - "exec1" -> "host1", - "exec1" -> "host1", - "exec2" -> "host2", - "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF) + "exec1" -> "host1", + "exec1" -> "host1", + "exec2" -> "host2", + "exec2" -> "host2")) { + val taskOption = manager.resourceOffer(k, v, NO_PREF)._1 assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1078,7 +1104,7 @@ class TaskSetManagerSuite assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1117,7 +1143,7 @@ class TaskSetManagerSuite "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF) + val taskOption = manager.resourceOffer(k, v, NO_PREF)._1 assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1150,7 +1176,7 @@ class TaskSetManagerSuite manager.handleFailedTask(task.taskId, TaskState.FAILED, endReason) sched.endedTasks(task.taskId) = endReason assert(!manager.isZombie) - val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF) + val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF)._1 assert(nextTask.isDefined, s"no offer for attempt $attempt of $index") tasks += nextTask.get } @@ -1166,7 +1192,7 @@ class TaskSetManagerSuite assert(manager.checkSpeculatableTasks(0)) assert(sched.speculativeTasks.toSet === Set(3, 4)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOption5.isDefined) val speculativeTask = taskOption5.get assert(speculativeTask.index === 3 || speculativeTask.index === 4) @@ -1191,7 +1217,7 @@ class TaskSetManagerSuite assert(!manager.isZombie) // now run another speculative task - val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOpt6.isDefined) val speculativeTask2 = taskOpt6.get assert(speculativeTask2.index === 3 || speculativeTask2.index === 4) @@ -1222,7 +1248,7 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock(1)) when(mockDAGScheduler.taskEnded(any(), any(), any(), any(), any(), any())).thenAnswer( (invocationOnMock: InvocationOnMock) => assert(manager.isZombie)) - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOption.isDefined) // this would fail, inside our mock dag scheduler, if it calls dagScheduler.taskEnded() too soon manager.handleSuccessfulTask(0, createTaskResult(0)) @@ -1267,7 +1293,7 @@ class TaskSetManagerSuite "exec2" -> "host1" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY)} + (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY)._1} } assert(taskDescs.size === 4) @@ -1304,7 +1330,7 @@ class TaskSetManagerSuite "exec2" -> "host2" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY)} + (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY)._1} } assert(taskDescs.size === 4) @@ -1340,7 +1366,7 @@ class TaskSetManagerSuite val taskSetManager = new TaskSetManager(sched, taskSet, 1, Some(blacklistTracker)) val taskSetManagerSpy = spy(taskSetManager) - val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) + val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY)._1 // Assert the task has been black listed on the executor it was last executed on. when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean(), anyBoolean())).thenAnswer( @@ -1368,9 +1394,9 @@ class TaskSetManagerSuite val manager1 = new TaskSetManager(sched, taskSet1, MAX_TASK_FAILURES, clock = new ManualClock) // all tasks from the first taskset have the same jars - val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF) + val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOption1.get.addedJars === addedJarsPreTaskSet) - val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF) + val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOption2.get.addedJars === addedJarsPreTaskSet) // even with a jar added mid-TaskSet @@ -1378,7 +1404,7 @@ class TaskSetManagerSuite sc.addJar(jarPath.toString) val addedJarsMidTaskSet = Map[String, Long](sc.addedJars.toSeq: _*) assert(addedJarsPreTaskSet !== addedJarsMidTaskSet) - val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF) + val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF)._1 // which should have the old version of the jars list assert(taskOption3.get.addedJars === addedJarsPreTaskSet) @@ -1386,7 +1412,7 @@ class TaskSetManagerSuite val taskSet2 = FakeTask.createTaskSet(1) val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, clock = new ManualClock) - val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF) + val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOption4.get.addedJars === addedJarsMidTaskSet) } @@ -1480,11 +1506,11 @@ class TaskSetManagerSuite } // Offer resources for 4 tasks to start for ((exec, host) <- Seq( - "exec1" -> "host1", - "exec1" -> "host1", - "exec3" -> "host3", - "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + "exec1" -> "host1", + "exec1" -> "host1", + "exec3" -> "host3", + "exec2" -> "host2")) { + val taskOption = manager.resourceOffer(exec, host, NO_PREF)._1 assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -1510,7 +1536,7 @@ class TaskSetManagerSuite assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY) + val taskOption = manager.resourceOffer("exec2", "host2", ANY)._1 assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -1552,11 +1578,11 @@ class TaskSetManagerSuite } // Offer resources for 4 tasks to start for ((k, v) <- List( - "exec1" -> "host1", - "exec1" -> "host1", - "exec2" -> "host2", - "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF) + "exec1" -> "host1", + "exec1" -> "host1", + "exec2" -> "host2", + "exec2" -> "host2")) { + val taskOption = manager.resourceOffer(k, v, NO_PREF)._1 assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1576,7 +1602,7 @@ class TaskSetManagerSuite assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF)._1 assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1636,7 +1662,7 @@ class TaskSetManagerSuite assert(FakeRackUtil.numBatchInvocation === 1) assert(FakeRackUtil.numSingleHostInvocation === 0) // with rack locality, reject an offer on a host with an unknown rack - assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL).isEmpty) + assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL)._1.isEmpty) (0 until 20).foreach { rackIdx => (0 until 5).foreach { offerIdx => // if we offer hosts which are not in preferred locations, @@ -1644,9 +1670,9 @@ class TaskSetManagerSuite // but accept them at RACK_LOCAL level if they're on OK racks val hostIdx = 100 + rackIdx assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.NODE_LOCAL) - .isEmpty) + ._1.isEmpty) assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL) - .isDefined) + ._1.isDefined) } } // check no more expensive calls to the rack resolution. manager.resourceOffer() will call @@ -1655,7 +1681,7 @@ class TaskSetManagerSuite assert(FakeRackUtil.numBatchInvocation === 1) } - test("TaskSetManager allocate resource addresses from available resources") { + test("TaskSetManager passes task resource along") { import TestUtils._ sc = new SparkContext("local", "test") @@ -1664,15 +1690,13 @@ class TaskSetManagerSuite val taskSet = FakeTask.createTaskSet(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - val availableResources = Map(GPU -> ArrayBuffer("0", "1", "2", "3")) - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, availableResources) + val taskResourceAssignments = Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))) + val taskOption = + manager.resourceOffer("exec1", "host1", NO_PREF, taskResourceAssignments)._1 assert(taskOption.isDefined) val allocatedResources = taskOption.get.resources assert(allocatedResources.size == 1) assert(allocatedResources(GPU).addresses sameElements Array("0", "1")) - // Allocated resource addresses should still present in `availableResources`, they will only - // get removed inside TaskSchedulerImpl later. - assert(availableResources(GPU) sameElements Array("0", "1", "2", "3")) } test("SPARK-26755 Ensure that a speculative task is submitted only once for execution") { @@ -1691,7 +1715,7 @@ class TaskSetManagerSuite // Offer resources for 4 tasks to start, 2 on each exec Seq("exec1" -> "host1", "exec2" -> "host2").foreach { case (exec, host) => (0 until 2).foreach { _ => - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + val taskOption = manager.resourceOffer(exec, host, NO_PREF)._1 assert(taskOption.isDefined) assert(taskOption.get.executorId === exec) } @@ -1715,8 +1739,8 @@ class TaskSetManagerSuite // Offer resource to start the speculative attempt for the running task. We offer more // resources, and ensure that speculative tasks get scheduled appropriately -- only one extra // copy per speculatable task - val taskOption2 = manager.resourceOffer("exec1", "host1", NO_PREF) - val taskOption3 = manager.resourceOffer("exec2", "host2", NO_PREF) + val taskOption2 = manager.resourceOffer("exec1", "host1", NO_PREF)._1 + val taskOption3 = manager.resourceOffer("exec2", "host2", NO_PREF)._1 assert(taskOption2.isDefined) val task2 = taskOption2.get // Ensure that task index 3 is launched on host1 and task index 4 on host2 @@ -1736,9 +1760,9 @@ class TaskSetManagerSuite assert(manager.copiesRunning(1) === 2) assert(manager.copiesRunning(3) === 2) // Offering additional resources should not lead to any speculative tasks being respawned - assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) - assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) - assert(manager.resourceOffer("exec3", "host3", ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.isEmpty) + assert(manager.resourceOffer("exec2", "host2", ANY)._1.isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY)._1.isEmpty) } test("SPARK-26755 Ensure that a speculative task obeys original locality preferences") { @@ -1761,7 +1785,7 @@ class TaskSetManagerSuite } // Offer resources for 3 tasks to start Seq("exec1" -> "host1", "exec2" -> "host2", "exec3" -> "host3").foreach { case (exec, host) => - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + val taskOption = manager.resourceOffer(exec, host, NO_PREF)._1 assert(taskOption.isDefined) assert(taskOption.get.executorId === exec) } @@ -1774,17 +1798,17 @@ class TaskSetManagerSuite assert(manager.checkSpeculatableTasks(0)) assert(sched.speculativeTasks.toSet === Set(0, 1)) // Ensure that the speculatable tasks obey the original locality preferences - assert(manager.resourceOffer("exec4", "host4", NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec4", "host4", NODE_LOCAL)._1.isEmpty) // task 1 does have a node-local preference for host2 -- but we've already got a regular // task running there, so we should not schedule a speculative there as well. - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL).isEmpty) - assert(manager.resourceOffer("exec3", "host3", NODE_LOCAL).isDefined) - assert(manager.resourceOffer("exec4", "host4", ANY).isDefined) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL)._1.isEmpty) + assert(manager.resourceOffer("exec3", "host3", NODE_LOCAL)._1.isDefined) + assert(manager.resourceOffer("exec4", "host4", ANY)._1.isDefined) // Since, all speculatable tasks have been launched, making another offer // should not schedule any more tasks - assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.isEmpty) assert(!manager.checkSpeculatableTasks(0)) - assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY)._1.isEmpty) } private def testSpeculationDurationSetup( @@ -1793,15 +1817,16 @@ class TaskSetManagerSuite numTasks: Int, numExecutorCores: Int, numCoresPerTask: Int): (TaskSetManager, ManualClock) = { - sc = new SparkContext("local", "test") - sc.conf.set(config.SPECULATION_ENABLED, true) - sc.conf.set(config.SPECULATION_QUANTILE.key, speculationQuantile.toString) + val conf = new SparkConf() + conf.set(config.SPECULATION_ENABLED, true) + conf.set(config.SPECULATION_QUANTILE.key, speculationQuantile.toString) // Set the number of slots per executor - sc.conf.set(config.EXECUTOR_CORES.key, numExecutorCores.toString) - sc.conf.set(config.CPUS_PER_TASK.key, numCoresPerTask.toString) + conf.set(config.EXECUTOR_CORES.key, numExecutorCores.toString) + conf.set(config.CPUS_PER_TASK.key, numCoresPerTask.toString) if (speculationThresholdOpt.isDefined) { - sc.conf.set(config.SPECULATION_TASK_DURATION_THRESHOLD.key, speculationThresholdOpt.get) + conf.set(config.SPECULATION_TASK_DURATION_THRESHOLD.key, speculationThresholdOpt.get) } + sc = new SparkContext("local", "test", conf) sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) // Create a task set with the given number of tasks val taskSet = FakeTask.createTaskSet(numTasks) @@ -1890,15 +1915,28 @@ class TaskSetManagerSuite test("SPARK-30417 when spark.task.cpus is greater than spark.executor.cores due to " + "standalone settings, speculate if there is only one task in the stage") { - val (manager, clock) = testSpeculationDurationSetup( - Some("60min"), - // Set the quantile to be 1.0 so that regular speculation would not be triggered - speculationQuantile = 1.0, - numTasks = 1, - numExecutorCores = 1, - numCoresPerTask = 2 - ) + val numTasks = 1 + val numCoresPerTask = 2 + val conf = new SparkConf() + // skip throwing exception when cores per task > cores per executor to emulate standalone mode + conf.set(SKIP_VALIDATE_CORES_TESTING, true) + conf.set(config.SPECULATION_ENABLED, true) + conf.set(config.SPECULATION_QUANTILE.key, "1.0") + // Skip setting cores per executor to emulate standalone default mode + conf.set(config.CPUS_PER_TASK.key, numCoresPerTask.toString) + conf.set(config.SPECULATION_TASK_DURATION_THRESHOLD.key, "60min") + sc = new SparkContext("local", "test", conf) + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + // Create a task set with the given number of tasks + val taskSet = FakeTask.createTaskSet(numTasks) + val clock = new ManualClock() + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + manager.isZombie = false + // Offer resources for the task to start + for (i <- 1 to numTasks) { + manager.resourceOffer(s"exec$i", s"host$i", NO_PREF) + } clock.advance(1000*60*60) assert(!manager.checkSpeculatableTasks(0)) assert(sched.speculativeTasks.size == 0) @@ -1915,7 +1953,7 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) assert(sched.taskSetsFailed.isEmpty) - val offerResult = manager.resourceOffer("exec1", "host1", ANY) + val offerResult = manager.resourceOffer("exec1", "host1", ANY)._1 assert(offerResult.isDefined, "Expect resource offer on iteration 0 to return a task") assert(offerResult.get.index === 0) @@ -1942,7 +1980,8 @@ class TaskSetManagerSuite TestUtils.waitUntilExecutorsUp(sc, 2, 60000) val tasks = Array.tabulate[Task[_]](2)(partition => new FakeLongTasks(stageId = 0, partition)) - val taskSet: TaskSet = new TaskSet(tasks, stageId = 0, stageAttemptId = 0, priority = 0, null) + val taskSet: TaskSet = new TaskSet(tasks, stageId = 0, stageAttemptId = 0, priority = 0, null, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val stageId = taskSet.stageId val stageAttemptId = taskSet.stageAttemptId sched.submitTasks(taskSet) diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala index 15733b0d932ec..8c6f86a6c0e88 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala @@ -70,13 +70,13 @@ class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] val execs = sched.getExecutorIds() execs.foreach(execId => sched.decommissionExecutor(execId)) - val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 2.seconds) + val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 10.seconds) assert(asyncCountResult === 10) // Try and launch task after decommissioning, this should fail val postDecommissioned = input.map(x => x) val postDecomAsyncCount = postDecommissioned.countAsync() val thrown = intercept[java.util.concurrent.TimeoutException]{ - val result = ThreadUtils.awaitResult(postDecomAsyncCount, 2.seconds) + val result = ThreadUtils.awaitResult(postDecomAsyncCount, 10.seconds) } assert(postDecomAsyncCount.isCompleted === false, "After exec decommission new task could not launch") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerMasterSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerMasterSuite.scala new file mode 100644 index 0000000000000..0d54726af7ee8 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerMasterSuite.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.junit.Assert.assertTrue + +import org.apache.spark.{SparkConf, SparkFunSuite} + +class BlockManagerMasterSuite extends SparkFunSuite { + + test("SPARK-31422: getMemoryStatus should not fail after BlockManagerMaster stops") { + val bmm = new BlockManagerMaster(null, null, new SparkConf, true) + assertTrue(bmm.getMemoryStatus.isEmpty) + } + + test("SPARK-31422: getStorageStatus should not fail after BlockManagerMaster stops") { + val bmm = new BlockManagerMaster(null, null, new SparkConf, true) + assertTrue(bmm.getStorageStatus.isEmpty) + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index ccc525e854838..c757dee43808d 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -51,7 +51,7 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B override def beforeEach(): Unit = { super.beforeEach() val conf = testConf.clone - conf.set("spark.local.dir", rootDirs).set("spark.diskStore.subDirectories", "1") + conf.set("spark.local.dir", rootDirs) diskBlockManager = new DiskBlockManager(conf, deleteFilesOnStop = true) } @@ -90,45 +90,4 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B for (i <- 0 until numBytes) writer.write(i) writer.close() } - - test("temporary shuffle/local file should be able to handle disk failures") { - try { - // the following two lines pre-create subdirectories under each root dir of block manager - diskBlockManager.getFile("1") - diskBlockManager.getFile("2") - - val tempShuffleFile1 = diskBlockManager.createTempShuffleBlock()._2 - val tempLocalFile1 = diskBlockManager.createTempLocalBlock()._2 - assert(tempShuffleFile1.exists(), "There are no bad disks, so temp shuffle file exists") - assert(tempLocalFile1.exists(), "There are no bad disks, so temp local file exists") - - // partial disks damaged - rootDir0.setExecutable(false) - val tempShuffleFile2 = diskBlockManager.createTempShuffleBlock()._2 - val tempLocalFile2 = diskBlockManager.createTempLocalBlock()._2 - // It's possible that after 10 retries we still not able to find the healthy disk. we need to - // remove the flakiness of these two asserts - if (tempShuffleFile2.getParentFile.getParentFile.getParent === rootDir1.getAbsolutePath) { - assert(tempShuffleFile2.exists(), - "There is only one bad disk, so temp shuffle file should be created") - } - if (tempLocalFile2.getParentFile.getParentFile.getParent === rootDir1.getAbsolutePath) { - assert(tempLocalFile2.exists(), - "There is only one bad disk, so temp local file should be created") - } - - // all disks damaged - rootDir1.setExecutable(false) - val tempShuffleFile3 = diskBlockManager.createTempShuffleBlock()._2 - val tempLocalFile3 = diskBlockManager.createTempLocalBlock()._2 - assert(!tempShuffleFile3.exists(), - "All disks are broken, so there should be no temp shuffle file created") - assert(!tempLocalFile3.exists(), - "All disks are broken, so there should be no temp local file created") - } finally { - rootDir0.setExecutable(true) - rootDir1.setExecutable(true) - } - - } } 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 9f0cdeac9ca39..757e03b2b50e5 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -47,7 +47,7 @@ import org.apache.spark.status.api.v1.{JacksonMessageWriter, RDDDataDistribution private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler { - private val cssWhiteList = List("bootstrap.min.css", "vis.min.css") + private val cssWhiteList = List("bootstrap.min.css", "vis-timeline-graph2d.min.css") private def isInWhileList(uri: String): Boolean = cssWhiteList.exists(uri.endsWith) diff --git a/dev/.rat-excludes b/dev/.rat-excludes index b1a84945d2a1f..473551f208994 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -31,8 +31,10 @@ d3.min.js dagre-d3.min.js graphlib-dot.min.js sorttable.js -vis.min.js -vis.min.css +vis-timeline-graph2d.min.js +vis-timeline-graph2d.min.js.map +vis-timeline-graph2d.min.css +vis-timeline-graph2d.min.css.map dataTables.bootstrap4.1.10.20.min.css dataTables.bootstrap4.1.10.20.min.js dataTables.rowsGroup.js diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh index cda21ebdcd4d8..4a003d7e07de2 100755 --- a/dev/create-release/do-release-docker.sh +++ b/dev/create-release/do-release-docker.sh @@ -93,10 +93,10 @@ done GPG_KEY_FILE="$WORKDIR/gpg.key" fcreate_secure "$GPG_KEY_FILE" -$GPG --export-secret-key --armor "$GPG_KEY" > "$GPG_KEY_FILE" +$GPG --export-secret-key --armor --pinentry-mode loopback --passphrase "$GPG_PASSPHRASE" "$GPG_KEY" > "$GPG_KEY_FILE" run_silent "Building spark-rm image with tag $IMGTAG..." "docker-build.log" \ - docker build --no-cache -t "spark-rm:$IMGTAG" --build-arg UID=$UID "$SELF/spark-rm" + docker build -t "spark-rm:$IMGTAG" --build-arg UID=$UID "$SELF/spark-rm" # Write the release information to a file with environment variables to be used when running the # image. diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index 1282e15e79913..8ee94a67f34fd 100755 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -159,10 +159,14 @@ function get_release_info { export SPARK_PACKAGE_VERSION="$RELEASE_TAG" # Gather some user information. - export ASF_USERNAME=$(read_config "ASF user" "$LOGNAME") + if [ -z "$ASF_USERNAME" ]; then + export ASF_USERNAME=$(read_config "ASF user" "$LOGNAME") + fi - GIT_NAME=$(git config user.name || echo "") - export GIT_NAME=$(read_config "Full name" "$GIT_NAME") + if [ -z "$GIT_NAME" ]; then + GIT_NAME=$(git config user.name || echo "") + export GIT_NAME=$(read_config "Full name" "$GIT_NAME") + fi export GIT_EMAIL="$ASF_USERNAME@apache.org" export GPG_KEY=$(read_config "GPG key" "$GIT_EMAIL") diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index d310aaf988891..540dc90f42817 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -20,9 +20,9 @@ # Includes: # * Java 8 # * Ivy -# * Python 3.7 -# * Ruby 2.7 +# * Python (2.7.15/3.6.7) # * R-base/R-base-dev (3.6.1) +# * Ruby 2.3 build utilities FROM ubuntu:18.04 @@ -38,6 +38,10 @@ ARG GEM_PKGS="jekyll:4.0.0 jekyll-redirect-from:0.16.0 rouge:3.15.0" # Install extra needed repos and refresh. # - CRAN repo +# - Ruby repo (for doc generation) +# +# This is all in a single "RUN" command so that if anything changes, "apt update" is run to fetch +# the most current package versions (instead of potentially using old versions cached by docker). RUN apt-get clean && apt-get update && $APT_INSTALL gnupg ca-certificates && \ echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran35/' >> /etc/apt/sources.list && \ gpg --keyserver keyserver.ubuntu.com --recv-key E298A3A825C0D65DFD57CBB651716619E084DAB9 && \ @@ -46,43 +50,33 @@ RUN apt-get clean && apt-get update && $APT_INSTALL gnupg ca-certificates && \ rm -rf /var/lib/apt/lists/* && \ apt-get clean && \ apt-get update && \ + $APT_INSTALL software-properties-common && \ + apt-add-repository -y ppa:brightbox/ruby-ng && \ + apt-get update && \ # Install openjdk 8. $APT_INSTALL openjdk-8-jdk && \ update-alternatives --set java /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java && \ # Install build / source control tools $APT_INSTALL curl wget git maven ivy subversion make gcc lsof libffi-dev \ - pandoc pandoc-citeproc libssl-dev libcurl4-openssl-dev libxml2-dev - -ENV PATH "$PATH:/root/.pyenv/bin:/root/.pyenv/shims" -RUN curl -L https://github.com/pyenv/pyenv-installer/raw/dd3f7d0914c5b4a416ca71ffabdf2954f2021596/bin/pyenv-installer | bash -RUN $APT_INSTALL libbz2-dev libreadline-dev libsqlite3-dev -RUN pyenv install 3.7.6 -RUN pyenv global 3.7.6 -RUN python --version -RUN pip install --upgrade pip -RUN pip --version -RUN pip install $PIP_PKGS - -ENV PATH "$PATH:/root/.rbenv/bin:/root/.rbenv/shims" -RUN curl -fsSL https://github.com/rbenv/rbenv-installer/raw/108c12307621a0aa06f19799641848dde1987deb/bin/rbenv-installer | bash -RUN rbenv install 2.7.0 -RUN rbenv global 2.7.0 -RUN ruby --version -RUN $APT_INSTALL g++ -RUN gem --version -RUN gem install --no-document $GEM_PKGS - -RUN \ + pandoc pandoc-citeproc libssl-dev libcurl4-openssl-dev libxml2-dev && \ curl -sL https://deb.nodesource.com/setup_11.x | bash && \ - $APT_INSTALL nodejs - -# Install R packages and dependencies used when building. -# R depends on pandoc*, libssl (which are installed above). -RUN \ + $APT_INSTALL nodejs && \ + # Install needed python packages. Use pip for installing packages (for consistency). + $APT_INSTALL libpython3-dev python3-pip python3-setuptools && \ + # Change default python version to python3. + update-alternatives --install /usr/bin/python python /usr/bin/python2.7 1 && \ + update-alternatives --install /usr/bin/python python /usr/bin/python3.6 2 && \ + update-alternatives --set python /usr/bin/python3.6 && \ + pip3 install $PIP_PKGS && \ + # Install R packages and dependencies used when building. + # R depends on pandoc*, libssl (which are installed above). $APT_INSTALL r-base r-base-dev && \ $APT_INSTALL texlive-latex-base texlive texlive-fonts-extra texinfo qpdf && \ Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" && \ - Rscript -e "devtools::install_github('jimhester/lintr')" + Rscript -e "devtools::install_github('jimhester/lintr')" && \ + # Install tools needed to build the documentation. + $APT_INSTALL ruby2.5 ruby2.5-dev && \ + gem install --no-document $GEM_PKGS WORKDIR /opt/spark-rm/output diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 6ab937ff40652..8f4d48fbad6c7 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -35,7 +35,7 @@ commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar commons-cli/1.2//commons-cli-1.2.jar commons-codec/1.10//commons-codec-1.10.jar commons-collections/3.2.2//commons-collections-3.2.2.jar -commons-compiler/3.0.15//commons-compiler-3.0.15.jar +commons-compiler/3.0.16//commons-compiler-3.0.16.jar commons-compress/1.8.1//commons-compress-1.8.1.jar commons-configuration/1.6//commons-configuration-1.6.jar commons-crypto/1.0.0//commons-crypto-1.0.0.jar @@ -105,7 +105,7 @@ jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar -janino/3.0.15//janino-3.0.15.jar +janino/3.0.16//janino-3.0.16.jar javassist/3.25.0-GA//javassist-3.25.0-GA.jar javax.inject/1//javax.inject-1.jar javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 6f563817ab2b6..3b1d3adbb8170 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -33,7 +33,7 @@ commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar commons-cli/1.2//commons-cli-1.2.jar commons-codec/1.10//commons-codec-1.10.jar commons-collections/3.2.2//commons-collections-3.2.2.jar -commons-compiler/3.0.15//commons-compiler-3.0.15.jar +commons-compiler/3.0.16//commons-compiler-3.0.16.jar commons-compress/1.8.1//commons-compress-1.8.1.jar commons-configuration/1.6//commons-configuration-1.6.jar commons-crypto/1.0.0//commons-crypto-1.0.0.jar @@ -118,7 +118,7 @@ jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar -janino/3.0.15//janino-3.0.15.jar +janino/3.0.16//janino-3.0.16.jar javassist/3.25.0-GA//javassist-3.25.0-GA.jar javax.inject/1//javax.inject-1.jar javax.jdo/3.2.0-m3//javax.jdo-3.2.0-m3.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 90cbd023d7664..3486ed1e477e9 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -30,7 +30,7 @@ commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar commons-cli/1.2//commons-cli-1.2.jar commons-codec/1.10//commons-codec-1.10.jar commons-collections/3.2.2//commons-collections-3.2.2.jar -commons-compiler/3.0.15//commons-compiler-3.0.15.jar +commons-compiler/3.0.16//commons-compiler-3.0.16.jar commons-compress/1.8.1//commons-compress-1.8.1.jar commons-configuration2/2.1.1//commons-configuration2-2.1.1.jar commons-crypto/1.0.0//commons-crypto-1.0.0.jar @@ -117,7 +117,7 @@ jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar -janino/3.0.15//janino-3.0.15.jar +janino/3.0.16//janino-3.0.16.jar javassist/3.25.0-GA//javassist-3.25.0-GA.jar javax.inject/1//javax.inject-1.jar javax.jdo/3.2.0-m3//javax.jdo-3.2.0-m3.jar diff --git a/dev/lint-python b/dev/lint-python index a9d2a0b75ac56..d5491f2447176 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -16,9 +16,6 @@ # limitations under the License. # # define test binaries + versions -PYDOCSTYLE_BUILD="pydocstyle" -MINIMUM_PYDOCSTYLE="3.0.0" - FLAKE8_BUILD="flake8" MINIMUM_FLAKE8="3.5.0" @@ -165,48 +162,6 @@ flake8 checks failed." fi } -function pydocstyle_test { - local PYDOCSTYLE_REPORT= - local PYDOCSTYLE_STATUS= - local PYDOCSTYLE_VERSION= - local EXPECTED_PYDOCSTYLE= - - # Exclude auto-generated configuration file. - local DOC_PATHS_TO_CHECK="$( cd "${SPARK_ROOT_DIR}" && find . -name "*.py" | grep -vF 'functions.py' )" - - # Check python document style, skip check if pydocstyle is not installed. - if ! hash "$PYDOCSTYLE_BUILD" 2> /dev/null; then - echo "The pydocstyle command was not found. Skipping pydocstyle checks for now." - echo - return - fi - - PYDOCSTYLE_VERSION="$($PYDOCSTYLE_BUILD --version)" - EXPECTED_PYDOCSTYLE="$(satisfies_min_version $PYDOCSTYLE_VERSION $MINIMUM_PYDOCSTYLE)" - - if [[ "$EXPECTED_PYDOCSTYLE" == "False" ]]; then - echo "\ -The minimum version of pydocstyle needs to be $MINIMUM_PYDOCSTYLE. -Your current version is $PYDOCSTYLE_VERSION. -Skipping pydocstyle checks for now." - echo - return - fi - - echo "starting $PYDOCSTYLE_BUILD test..." - PYDOCSTYLE_REPORT=$( ($PYDOCSTYLE_BUILD --config=dev/tox.ini $DOC_PATHS_TO_CHECK) 2>&1) - PYDOCSTYLE_STATUS=$? - - if [ "$PYDOCSTYLE_STATUS" -ne 0 ]; then - echo "pydocstyle checks failed:" - echo "$PYDOCSTYLE_REPORT" - exit "$PYDOCSTYLE_STATUS" - else - echo "pydocstyle checks passed." - echo - fi -} - function sphinx_test { local SPHINX_REPORT= local SPHINX_STATUS= @@ -252,7 +207,6 @@ PYTHON_SOURCE="$(find . -name "*.py")" compile_python_test "$PYTHON_SOURCE" pycodestyle_test "$PYTHON_SOURCE" flake8_test -pydocstyle_test sphinx_test echo diff --git a/dev/tox.ini b/dev/tox.ini index 54f65692c8303..3ee13c09b5de7 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -17,5 +17,3 @@ ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 max-line-length=100 exclude=cloudpickle.py,heapq3.py,shared.py,python/docs/conf.py,work/*/*.py,python/.eggs/*,dist/*,.git/* -[pydocstyle] -ignore=D100,D101,D102,D103,D104,D105,D106,D107,D200,D201,D202,D203,D204,D205,D206,D207,D208,D209,D210,D211,D212,D213,D214,D215,D300,D301,D302,D400,D401,D402,D403,D404,D405,D406,D407,D408,D409,D410,D411,D412,D413,D414 diff --git a/docs/README.md b/docs/README.md index c16f67c2c8175..22039871cf63d 100644 --- a/docs/README.md +++ b/docs/README.md @@ -31,49 +31,19 @@ whichever version of Spark you currently have checked out of revision control. The Spark documentation build uses a number of tools to build HTML docs and API docs in Scala, Java, Python, R and SQL. -You need to have Ruby 2 (preferably Ruby 2.6+) and Python 3 (preferably Python 3.7+) installed. - -You'll also need to install the following libraries: - -```sh -gem install jekyll:4.0.0 jekyll-redirect-from:0.16.0 rouge:3.15.0 -``` - -### Using rbenv and pyenv - -A handy way to install and manage various versions of Ruby and Python is with [`rbenv`] and [`pyenv`]. - -[`rbenv`]: https://github.com/rbenv/rbenv -[`pyenv`]: https://github.com/pyenv/pyenv - -On macOS you can install them with Homebrew: +You need to have [Ruby](https://www.ruby-lang.org/en/documentation/installation/) and +[Python](https://docs.python.org/2/using/unix.html#getting-and-installing-the-latest-version-of-python) +installed. Also install the following libraries: ```sh -brew install rbenv pyenv +$ sudo gem install jekyll jekyll-redirect-from rouge ``` -To activate them, you'll need to run these commands or add them to the end of your `.bash_profile`: - -```sh -eval "$(rbenv init -)" -eval "$(pyenv init -)" -``` - -You can now use them to install specific versions of Ruby and Python and associate them with -the Spark home directory. Whenever you navigate to this directory or any of its subdirectories, these versions of Ruby and Python will be automatically activated. - -```sh -rbenv install 2.7.0 -pyenv install 3.7.6 - -cd /path/to/spark/root -rbenv local 2.7.0 -pyenv local 3.7.6 -``` +Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to replace gem with gem2.0. ### R Documentation -If you'd like to generate R documentation, you'll need to install R, [install Pandoc](https://pandoc.org/installing.html), +If you'd like to generate R documentation, you'll need to [install Pandoc](https://pandoc.org/installing.html) and install these libraries: ```sh @@ -88,7 +58,7 @@ Note: Other versions of roxygen2 might work in SparkR documentation generation b To generate API docs for any language, you'll need to install these libraries: ```sh -pip install sphinx==2.3.1 mkdocs==1.0.4 numpy==1.18.1 +$ sudo pip install sphinx mkdocs numpy ``` ## Generating the Documentation HTML diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index c17bfd371f717..badb98d2e593b 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -67,6 +67,8 @@ url: sql-pyspark-pandas-with-arrow.html#enabling-for-conversion-tofrom-pandas - text: "Pandas UDFs (a.k.a. Vectorized UDFs)" url: sql-pyspark-pandas-with-arrow.html#pandas-udfs-aka-vectorized-udfs + - text: "Pandas Function APIs" + url: sql-pyspark-pandas-with-arrow.html#pandas-function-apis - text: Usage Notes url: sql-pyspark-pandas-with-arrow.html#usage-notes - text: Migration Guide @@ -121,6 +123,8 @@ url: sql-ref-syntax-ddl-truncate-table.html - text: REPAIR TABLE url: sql-ref-syntax-ddl-repair-table.html + - text: USE DATABASE + url: sql-ref-syntax-qry-select-usedb.html - text: Data Manipulation Statements url: sql-ref-syntax-dml.html subitems: @@ -150,8 +154,18 @@ url: sql-ref-syntax-qry-select-distribute-by.html - text: LIMIT Clause url: sql-ref-syntax-qry-select-limit.html - - text: USE database - url: sql-ref-syntax-qry-select-usedb.html + - text: JOIN + url: sql-ref-syntax-qry-select-join.html + - text: Join Hints + url: sql-ref-syntax-qry-select-hints.html + - text: Set Operators + url: sql-ref-syntax-qry-select-setops.html + - text: TABLESAMPLE + url: sql-ref-syntax-qry-sampling.html + - text: Table-valued Function + url: sql-ref-syntax-qry-select-tvf.html + - text: Inline Table + url: sql-ref-syntax-qry-select-inline-table.html - text: EXPLAIN url: sql-ref-syntax-qry-explain.html - text: Auxiliary Statements @@ -174,7 +188,7 @@ - text: REFRESH TABLE url: sql-ref-syntax-aux-refresh-table.html - text: REFRESH - url: sql-ref-syntax-aux-cache-refresh.md + url: sql-ref-syntax-aux-cache-refresh.html - text: DESCRIBE url: sql-ref-syntax-aux-describe.html subitems: @@ -205,6 +219,8 @@ url: sql-ref-syntax-aux-show-partitions.html - text: SHOW CREATE TABLE url: sql-ref-syntax-aux-show-create-table.html + - text: SHOW VIEWS + url: sql-ref-syntax-aux-show-views.html - text: CONFIGURATION MANAGEMENT url: sql-ref-syntax-aux-conf-mgmt.html subitems: @@ -223,5 +239,14 @@ url: sql-ref-syntax-aux-resource-mgmt-list-file.html - text: LIST JAR url: sql-ref-syntax-aux-resource-mgmt-list-jar.html + - text: Functions + url: sql-ref-functions.html + subitems: + - text: Scalar UDFs (User-Defined Functions) + url: sql-ref-functions-udf-scalar.html + - text: UDAFs (User-Defined Aggregate Functions) + url: sql-ref-functions-udf-aggregate.html + - text: Integration with Hive UDFs/UDAFs/UDTFs + url: sql-ref-functions-udf-hive.html - text: Datetime Pattern url: sql-ref-datetime-pattern.html diff --git a/docs/configuration.md b/docs/configuration.md index 74d998ce53c43..676ecf5a82d48 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -143,6 +143,7 @@ of the most common options to set are: The name of your application. This will appear in the UI and in log data. + 0.9.0 spark.driver.cores @@ -163,6 +164,7 @@ of the most common options to set are: and memory overhead of objects in JVM). Setting a proper limit can protect the driver from out-of-memory errors. + 1.2.0 spark.driver.memory @@ -205,6 +207,7 @@ of the most common options to set are: spark.driver.resource.{resourceName}.discoveryScript for the driver to find the resource on startup. + 3.0.0 spark.driver.resource.{resourceName}.discoveryScript @@ -215,6 +218,7 @@ of the most common options to set are: name and an array of addresses. For a client-submitted driver, discovery script must assign different resource addresses to this driver comparing to other drivers on the same host. + 3.0.0 spark.driver.resource.{resourceName}.vendor @@ -225,6 +229,7 @@ of the most common options to set are: the Kubernetes device plugin naming convention. (e.g. For GPUs on Kubernetes this config would be set to nvidia.com or amd.com) + 3.0.0 spark.resources.discoveryPlugin @@ -292,6 +297,7 @@ of the most common options to set are: spark.executor.resource.{resourceName}.discoveryScript for the executor to find the resource on startup. + 3.0.0 spark.executor.resource.{resourceName}.discoveryScript @@ -301,6 +307,7 @@ of the most common options to set are: write to STDOUT a JSON string in the format of the ResourceInformation class. This has a name and an array of addresses. + 3.0.0 spark.executor.resource.{resourceName}.vendor @@ -311,6 +318,7 @@ of the most common options to set are: the Kubernetes device plugin naming convention. (e.g. For GPUs on Kubernetes this config would be set to nvidia.com or amd.com) + 3.0.0 spark.extraListeners @@ -322,6 +330,7 @@ of the most common options to set are: will be called; otherwise, a zero-argument constructor will be called. If no valid constructor can be found, the SparkContext creation will fail with an exception. + 1.3.0 spark.local.dir @@ -335,6 +344,7 @@ of the most common options to set are: Note: This will be overridden by SPARK_LOCAL_DIRS (Standalone), MESOS_SANDBOX (Mesos) or LOCAL_DIRS (YARN) environment variables set by the cluster manager. + 0.5.0 spark.logConf @@ -342,6 +352,7 @@ of the most common options to set are: Logs the effective SparkConf as INFO when a SparkContext is started. + 0.9.0 spark.master @@ -350,6 +361,7 @@ of the most common options to set are: The cluster manager to connect to. See the list of allowed master URL's. + 0.9.0 spark.submit.deployMode @@ -359,6 +371,7 @@ of the most common options to set are: Which means to launch driver program locally ("client") or remotely ("cluster") on one of the nodes inside the cluster. + 1.5.0 spark.log.callerContext @@ -368,6 +381,7 @@ of the most common options to set are: Its length depends on the Hadoop configuration hadoop.caller.context.max.size. It should be concise, and typically can have up to 50 characters. + 2.2.0 spark.driver.supervise @@ -376,6 +390,7 @@ of the most common options to set are: If true, restarts the driver automatically if it fails with a non-zero exit status. Only has effect in Spark standalone mode or Mesos cluster deploy mode. + 1.3.0 spark.driver.log.dfsDir @@ -462,6 +477,7 @@ Apart from these, the following properties are also available, and may be useful Instead, please set this through the --driver-java-options command line option or in your default properties file. + 3.0.0 spark.driver.extraJavaOptions @@ -535,6 +551,7 @@ Apart from these, the following properties are also available, and may be useful verbose gc logging to a file named for the executor ID of the app in /tmp, pass a 'value' of: -verbose:gc -Xloggc:/tmp/{{APP_ID}}-{{EXECUTOR_ID}}.gc + 3.0.0 spark.executor.extraJavaOptions @@ -571,6 +588,7 @@ Apart from these, the following properties are also available, and may be useful Sets the number of latest rolling log files that are going to be retained by the system. Older log files will be deleted. Disabled by default. + 1.1.0 spark.executor.logs.rolling.enableCompression @@ -579,6 +597,7 @@ Apart from these, the following properties are also available, and may be useful Enable executor log compression. If it is enabled, the rolled executor logs will be compressed. Disabled by default. + 2.0.2 spark.executor.logs.rolling.maxSize @@ -588,6 +607,7 @@ Apart from these, the following properties are also available, and may be useful Rolling is disabled by default. See spark.executor.logs.rolling.maxRetainedFiles for automatic cleaning of old logs. + 1.4.0 spark.executor.logs.rolling.strategy @@ -599,6 +619,7 @@ Apart from these, the following properties are also available, and may be useful For "size", use spark.executor.logs.rolling.maxSize to set the maximum file size for rolling. + 1.1.0 spark.executor.logs.rolling.time.interval @@ -609,6 +630,7 @@ Apart from these, the following properties are also available, and may be useful any interval in seconds. See spark.executor.logs.rolling.maxRetainedFiles for automatic cleaning of old logs. + 1.1.0 spark.executor.userClassPathFirst @@ -626,6 +648,7 @@ Apart from these, the following properties are also available, and may be useful Add the environment variable specified by EnvironmentVariableName to the Executor process. The user can specify multiple of these to set multiple environment variables. + 0.9.0 spark.redaction.regex @@ -635,6 +658,7 @@ Apart from these, the following properties are also available, and may be useful executor environments contain sensitive information. When this regex matches a property key or value, the value is redacted from the environment UI and various logs like YARN and event logs. + 2.1.2 spark.python.profile @@ -648,6 +672,7 @@ Apart from these, the following properties are also available, and may be useful By default the pyspark.profiler.BasicProfiler will be used, but this can be overridden by passing a profiler class in as a parameter to the SparkContext constructor. + 1.2.0 spark.python.profile.dump @@ -658,6 +683,7 @@ Apart from these, the following properties are also available, and may be useful by pstats.Stats(). If this is specified, the profile result will not be displayed automatically. + 1.2.0 spark.python.worker.memory @@ -668,6 +694,7 @@ Apart from these, the following properties are also available, and may be useful (e.g. 512m, 2g). If the memory used during aggregation goes above this amount, it will spill the data into disks. + 1.1.0 spark.python.worker.reuse @@ -686,6 +713,7 @@ Apart from these, the following properties are also available, and may be useful Comma-separated list of files to be placed in the working directory of each executor. Globs are allowed. + 1.0.0 spark.submit.pyFiles @@ -693,6 +721,7 @@ Apart from these, the following properties are also available, and may be useful Comma-separated list of .zip, .egg, or .py files to place on the PYTHONPATH for Python apps. Globs are allowed. + 1.0.1 spark.jars @@ -700,6 +729,7 @@ Apart from these, the following properties are also available, and may be useful Comma-separated list of jars to include on the driver and executor classpaths. Globs are allowed. + 0.9.0 spark.jars.packages @@ -712,6 +742,7 @@ Apart from these, the following properties are also available, and may be useful repositories given by the command-line option --repositories. For more details, see Advanced Dependency Management. + 1.5.0 spark.jars.excludes @@ -720,6 +751,7 @@ Apart from these, the following properties are also available, and may be useful Comma-separated list of groupId:artifactId, to exclude while resolving the dependencies provided in spark.jars.packages to avoid dependency conflicts. + 1.5.0 spark.jars.ivy @@ -729,6 +761,7 @@ Apart from these, the following properties are also available, and may be useful spark.jars.packages. This will override the Ivy property ivy.default.ivy.user.dir which defaults to ~/.ivy2. + 1.3.0 spark.jars.ivySettings @@ -741,6 +774,7 @@ Apart from these, the following properties are also available, and may be useful artifact server like Artifactory. Details on the settings file format can be found at Settings Files + 2.2.0 spark.jars.repositories @@ -749,6 +783,7 @@ Apart from these, the following properties are also available, and may be useful Comma-separated list of additional remote repositories to search for the maven coordinates given with --packages or spark.jars.packages. + 2.3.0 spark.pyspark.driver.python @@ -757,6 +792,7 @@ Apart from these, the following properties are also available, and may be useful Python binary executable to use for PySpark in driver. (default is spark.pyspark.python) + 2.1.0 spark.pyspark.python @@ -764,13 +800,14 @@ Apart from these, the following properties are also available, and may be useful Python binary executable to use for PySpark in both driver and executors. + 2.1.0 ### Shuffle Behavior - + @@ -780,6 +817,7 @@ Apart from these, the following properties are also available, and may be useful represents a fixed memory overhead per reduce task, so keep it small unless you have a large amount of memory. + @@ -790,6 +828,7 @@ Apart from these, the following properties are also available, and may be useful of inbound connections to one or more nodes, causing the workers to fail under load. By allowing it to limit the number of fetch requests, this scenario can be mitigated. + @@ -801,6 +840,7 @@ Apart from these, the following properties are also available, and may be useful is especially useful to reduce the load on the Node Manager when external shuffle is enabled. You can mitigate this issue by setting it to a lower value. + @@ -809,6 +849,7 @@ Apart from these, the following properties are also available, and may be useful Whether to compress map output files. Generally a good idea. Compression will use spark.io.compression.codec. + @@ -818,6 +859,7 @@ Apart from these, the following properties are also available, and may be useful specified. These buffers reduce the number of disk seeks and system calls made in creating intermediate shuffle files. + @@ -827,6 +869,7 @@ Apart from these, the following properties are also available, and may be useful set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC pauses or transient network connectivity issues. + @@ -836,6 +879,7 @@ Apart from these, the following properties are also available, and may be useful large clusters. For clusters with many hard disks and few hosts, this may result in insufficient concurrency to saturate all disks, and so users may consider increasing this value. + @@ -845,6 +889,7 @@ Apart from these, the following properties are also available, and may be useful block transfer. For environments where off-heap memory is tightly limited, users may wish to turn this off to force all allocations from Netty to be on-heap. + @@ -853,6 +898,7 @@ Apart from these, the following properties are also available, and may be useful (Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying is 15 seconds by default, calculated as maxRetries * retryWait. + @@ -865,6 +911,7 @@ Apart from these, the following properties are also available, and may be useful application (see spark.shuffle.service.enabled option below). If set below 1, will fallback to OS default defined by Netty's io.netty.util.NetUtil#SOMAXCONN. + @@ -893,6 +940,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -904,6 +952,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.retryWait), if those limits are reached the task will fail with fetch failure. + @@ -912,6 +961,7 @@ Apart from these, the following properties are also available, and may be useful (Advanced) In the sort-based shuffle manager, avoid merge-sorting data if there is no map-side aggregation and there are at most this many reduce partitions. + @@ -920,6 +970,7 @@ Apart from these, the following properties are also available, and may be useful Whether to compress data spilled during shuffles. Compression will use spark.io.compression.codec. + @@ -929,6 +980,7 @@ Apart from these, the following properties are also available, and may be useful accurately recorded. This helps to prevent OOM by avoiding underestimating shuffle block size when fetch shuffle blocks. + @@ -936,6 +988,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -943,6 +996,7 @@ Apart from these, the following properties are also available, and may be useful +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.reducer.maxSizeInFlight 48m1.4.0
    spark.reducer.maxReqsInFlight2.0.0
    spark.reducer.maxBlocksInFlightPerAddress2.2.1
    spark.shuffle.compress0.6.0
    spark.shuffle.file.buffer1.4.0
    spark.shuffle.io.maxRetries1.2.0
    spark.shuffle.io.numConnectionsPerPeer1.2.1
    spark.shuffle.io.preferDirectBufs1.2.0
    spark.shuffle.io.retryWait1.2.1
    spark.shuffle.io.backLog1.1.1
    spark.shuffle.service.enabled Cache entries limited to the specified memory footprint, in bytes unless otherwise specified. 2.3.0
    spark.shuffle.maxChunksBeingTransferred2.3.0
    spark.shuffle.sort.bypassMergeThreshold1.1.1
    spark.shuffle.spill.compress0.9.0
    spark.shuffle.accurateBlockThreshold2.2.1
    spark.shuffle.registration.timeout Timeout in milliseconds for registration to the external shuffle service. 2.3.0
    spark.shuffle.registration.maxAttempts When we fail to register to the external shuffle service, we will retry for maxAttempts times. 2.3.0
    @@ -982,6 +1036,7 @@ Apart from these, the following properties are also available, and may be useful The codec to compress logged events. If this is not given, spark.io.compression.codec will be used. + 3.0.0 spark.eventLog.erasureCoding.enabled @@ -1205,6 +1260,7 @@ Apart from these, the following properties are also available, and may be useful How many finished executions the Spark UI and status APIs remember before garbage collecting. + 1.5.0 spark.streaming.ui.retainedBatches @@ -1212,6 +1268,7 @@ Apart from these, the following properties are also available, and may be useful How many finished batches the Spark UI and status APIs remember before garbage collecting. + 1.0.0 spark.ui.retainedDeadExecutors @@ -1261,6 +1318,7 @@ Apart from these, the following properties are also available, and may be useful Whether to compress broadcast variables before sending them. Generally a good idea. Compression will use spark.io.compression.codec. + 0.6.0 spark.checkpoint.compress @@ -1268,7 +1326,8 @@ Apart from these, the following properties are also available, and may be useful Whether to compress RDD checkpoints. Generally a good idea. Compression will use spark.io.compression.codec. - + + 2.2.0 spark.io.compression.codec @@ -1283,6 +1342,7 @@ Apart from these, the following properties are also available, and may be useful org.apache.spark.io.SnappyCompressionCodec, and org.apache.spark.io.ZStdCompressionCodec. + 0.8.0 spark.io.compression.lz4.blockSize @@ -1292,6 +1352,7 @@ Apart from these, the following properties are also available, and may be useful is used. Lowering this block size will also lower shuffle memory usage when LZ4 is used. Default unit is bytes, unless otherwise specified. + 1.4.0 spark.io.compression.snappy.blockSize @@ -1301,6 +1362,7 @@ Apart from these, the following properties are also available, and may be useful Lowering this block size will also lower shuffle memory usage when Snappy is used. Default unit is bytes, unless otherwise specified. + 1.4.0 spark.io.compression.zstd.level @@ -1309,6 +1371,7 @@ Apart from these, the following properties are also available, and may be useful Compression level for Zstd compression codec. Increasing the compression level will result in better compression at the expense of more CPU and memory. + 2.3.0 spark.io.compression.zstd.bufferSize @@ -1318,6 +1381,7 @@ Apart from these, the following properties are also available, and may be useful is used. Lowering this size will lower the shuffle memory usage when Zstd is used, but it might increase the compression cost because of excessive JNI call overhead. + 2.3.0 spark.kryo.classesToRegister @@ -1327,7 +1391,7 @@ Apart from these, the following properties are also available, and may be useful with Kryo. See the tuning guide for more details. - 1.2.0 + 1.2.0 spark.kryo.referenceTracking @@ -1338,7 +1402,7 @@ Apart from these, the following properties are also available, and may be useful copies of the same object. Can be disabled to improve performance if you know this is not the case. - 0.8.0 + 0.8.0 spark.kryo.registrationRequired @@ -1350,7 +1414,7 @@ Apart from these, the following properties are also available, and may be useful significant performance overhead, so enabling this option can enforce strictly that a user has not omitted classes from registration. - 1.1.0 + 1.1.0 spark.kryo.registrator @@ -1364,7 +1428,7 @@ Apart from these, the following properties are also available, and may be useful KryoRegistrator. See the tuning guide for more details. - 0.5.0 + 0.5.0 spark.kryo.unsafe @@ -1373,7 +1437,7 @@ Apart from these, the following properties are also available, and may be useful Whether to use unsafe based Kryo serializer. Can be substantially faster by using Unsafe Based IO. - 2.1.0 + 2.1.0 spark.kryoserializer.buffer.max @@ -1383,7 +1447,7 @@ Apart from these, the following properties are also available, and may be useful This must be larger than any object you attempt to serialize and must be less than 2048m. Increase this if you get a "buffer limit exceeded" exception inside Kryo. - 1.4.0 + 1.4.0 spark.kryoserializer.buffer @@ -1393,7 +1457,7 @@ Apart from these, the following properties are also available, and may be useful Note that there will be one buffer per core on each worker. This buffer will grow up to spark.kryoserializer.buffer.max if needed. - 1.4.0 + 1.4.0 spark.rdd.compress @@ -1405,6 +1469,7 @@ Apart from these, the following properties are also available, and may be useful Can save substantial space at the cost of some extra CPU time. Compression will use spark.io.compression.codec. + 0.6.0 spark.serializer @@ -1420,6 +1485,7 @@ Apart from these, the following properties are also available, and may be useful org.apache.spark.Serializer. + 0.5.0 spark.serializer.objectStreamReset @@ -1431,13 +1497,14 @@ Apart from these, the following properties are also available, and may be useful objects to be collected. To turn off this periodic reset set it to -1. By default it will reset the serializer every 100 objects. + 1.0.0 ### Memory Management - + @@ -1504,6 +1571,7 @@ Apart from these, the following properties are also available, and may be useful on the driver, this may happen very occasionally or not at all. Not cleaning at all may lead to executors running out of disk space after a while. + @@ -1511,6 +1579,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -1519,6 +1588,7 @@ Apart from these, the following properties are also available, and may be useful Controls whether the cleaning thread should block on cleanup tasks (other than shuffle, which is controlled by spark.cleaner.referenceTracking.blocking.shuffle Spark property). + @@ -1526,6 +1596,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -1533,13 +1604,14 @@ Apart from these, the following properties are also available, and may be useful +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.memory.fraction 0.61.6.0
    spark.cleaner.referenceTracking Enables or disables context cleaning. 1.0.0
    spark.cleaner.referenceTracking.blocking1.0.0
    spark.cleaner.referenceTracking.blocking.shuffle Controls whether the cleaning thread should block on shuffle cleanup tasks. 1.1.1
    spark.cleaner.referenceTracking.cleanCheckpoints Controls whether to clean checkpoint files if the reference is out of scope. 1.4.0
    ### Execution Behavior - + @@ -1548,6 +1620,7 @@ Apart from these, the following properties are also available, and may be useful specified. Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit. + @@ -1557,6 +1630,7 @@ Apart from these, the following properties are also available, and may be useful help detect corrupted blocks, at the cost of computing and sending a little more data. It's possible to disable it if the network has other mechanisms to guarantee data won't be corrupted during broadcast. + @@ -1588,6 +1662,7 @@ Apart from these, the following properties are also available, and may be useful Default number of partitions in RDDs returned by transformations like join, reduceByKey, and parallelize when not set by user. + @@ -1607,6 +1682,7 @@ Apart from these, the following properties are also available, and may be useful Communication timeout to use when fetching files added through SparkContext.addFile() from the driver. + @@ -1619,6 +1695,7 @@ Apart from these, the following properties are also available, and may be useful disabled in order to use Spark local directories that reside on NFS filesystems (see SPARK-6313 for more details). + @@ -1627,6 +1704,7 @@ Apart from these, the following properties are also available, and may be useful Whether to overwrite files added through SparkContext.addFile() when the target file exists and its contents do not match those of the source. + @@ -1634,6 +1712,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -1643,25 +1722,32 @@ Apart from these, the following properties are also available, and may be useful time. This is used when putting multiple files into a partition. It is better to overestimate, then the partitions with small files will be faster than partitions with bigger files. + - - - + + + are not affected by these issues. + + - - - + + + output directories. We recommend that users do not disable this except if trying to achieve compatibility + with previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since data may + need to be rewritten to pre-existing output directories during checkpoint recovery. + + @@ -1681,6 +1767,7 @@ Apart from these, the following properties are also available, and may be useful Version 2 may have better performance, but version 1 may handle failures better in certain situations, as per MAPREDUCE-4815. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.broadcast.blockSize 4m0.5.0
    spark.broadcast.checksum2.1.1
    spark.executor.cores0.5.0
    spark.executor.heartbeatInterval1.0.0
    spark.files.useFetchCache1.2.2
    spark.files.overwrite1.0.0
    spark.files.maxPartitionBytes The maximum number of bytes to pack into a single partition when reading files. 2.1.0
    spark.files.openCostInBytes2.1.0
    spark.hadoop.cloneConffalseIf set to true, clones a new Hadoop Configuration object for each task. This + spark.hadoop.cloneConffalse + If set to true, clones a new Hadoop Configuration object for each task. This option should be enabled to work around Configuration thread-safety issues (see SPARK-2546 for more details). This is disabled by default in order to avoid unexpected performance regressions for jobs that - are not affected by these issues.1.0.3
    spark.hadoop.validateOutputSpecstrueIf set to true, validates the output specification (e.g. checking if the output directory already exists) + spark.hadoop.validateOutputSpecstrue + If set to true, validates the output specification (e.g. checking if the output directory already exists) used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing - output directories. We recommend that users do not disable this except if trying to achieve compatibility with - previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. - This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since - data may need to be rewritten to pre-existing output directories during checkpoint recovery.1.0.1
    spark.storage.memoryMapThreshold2.2.0
    @@ -1744,7 +1831,7 @@ Apart from these, the following properties are also available, and may be useful Port for all block managers to listen on. These exist on both the driver and the executors. - + 1.1.0 spark.driver.blockManager.port @@ -1753,7 +1840,7 @@ Apart from these, the following properties are also available, and may be useful Driver-specific port for the block manager to listen on, for cases where it cannot use the same configuration as executors. - + 2.1.0 spark.driver.bindAddress @@ -1767,7 +1854,7 @@ Apart from these, the following properties are also available, and may be useful the different ports used by the driver (RPC, block manager and UI) need to be forwarded from the container's host. - + 2.1.0 spark.driver.host @@ -1776,7 +1863,7 @@ Apart from these, the following properties are also available, and may be useful Hostname or IP address for the driver. This is used for communicating with the executors and the standalone Master. - + 0.7.0 spark.driver.port @@ -1785,7 +1872,7 @@ Apart from these, the following properties are also available, and may be useful Port for the driver to listen on. This is used for communicating with the executors and the standalone Master. - + 0.7.0 spark.rpc.io.backLog @@ -1795,7 +1882,7 @@ Apart from these, the following properties are also available, and may be useful need to be increased, so that incoming connections are not dropped when a large number of connections arrives in a short period of time. - + 3.0.0 spark.network.timeout @@ -1818,7 +1905,7 @@ Apart from these, the following properties are also available, and may be useful block transfer. For environments where off-heap memory is tightly limited, users may wish to turn this off to force all allocations to be on-heap. - + 3.0.0 spark.port.maxRetries @@ -1830,7 +1917,7 @@ Apart from these, the following properties are also available, and may be useful essentially allows it to try a range of ports from the start port specified to port + maxRetries. - + 1.1.1 spark.rpc.numRetries @@ -1873,7 +1960,7 @@ Apart from these, the following properties are also available, and may be useful out and giving up. To avoid unwilling timeout caused by long pause like GC, you can set larger value. - + 1.1.1 spark.network.maxRemoteBlockSizeFetchToMem @@ -1885,14 +1972,14 @@ Apart from these, the following properties are also available, and may be useful For users who enabled external shuffle service, this feature can only work when external shuffle service is at least 2.3.0. - + 3.0.0 ### Scheduling - + @@ -1926,6 +2013,7 @@ Apart from these, the following properties are also available, and may be useful Customize the locality wait for node locality. For example, you can set this to 0 to skip node locality and search immediately for rack locality (if your cluster has rack information). + @@ -1934,6 +2022,7 @@ Apart from these, the following properties are also available, and may be useful Customize the locality wait for process locality. This affects tasks that attempt to access cached data in a particular executor process. + @@ -1941,6 +2030,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -1948,6 +2038,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -1961,6 +2052,7 @@ Apart from these, the following properties are also available, and may be useful the maximum amount of time it will wait before scheduling begins is controlled by config spark.scheduler.maxRegisteredResourcesWaitingTime. + @@ -1971,6 +2063,7 @@ Apart from these, the following properties are also available, and may be useful to use fair sharing instead of queueing jobs one after another. Useful for multi-user services. + @@ -1978,6 +2071,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -1999,6 +2093,7 @@ Apart from these, the following properties are also available, and may be useful that register to the listener bus. Consider increasing value, if the listener events corresponding to shared queue are dropped. Increasing this value may result in the driver using more memory. + @@ -2008,6 +2103,7 @@ Apart from these, the following properties are also available, and may be useful Consider increasing value, if the listener events corresponding to appStatus queue are dropped. Increasing this value may result in the driver using more memory. + @@ -2017,6 +2113,7 @@ Apart from these, the following properties are also available, and may be useful executor management listeners. Consider increasing value if the listener events corresponding to executorManagement queue are dropped. Increasing this value may result in the driver using more memory. + @@ -2026,6 +2123,7 @@ Apart from these, the following properties are also available, and may be useful that write events to eventLogs. Consider increasing value if the listener events corresponding to eventLog queue are dropped. Increasing this value may result in the driver using more memory. + @@ -2035,6 +2133,18 @@ Apart from these, the following properties are also available, and may be useful Consider increasing value if the listener events corresponding to streams queue are dropped. Increasing this value may result in the driver using more memory. + + + + + + + @@ -2043,6 +2153,7 @@ Apart from these, the following properties are also available, and may be useful The timeout in seconds to wait to acquire a new executor and schedule a task before aborting a TaskSet which is unschedulable because of being completely blacklisted. + @@ -2153,6 +2264,7 @@ Apart from these, the following properties are also available, and may be useful If set to "true", performs speculative execution of tasks. This means if one or more tasks are running slowly in a stage, they will be re-launched. + @@ -2160,6 +2272,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -2167,6 +2280,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -2174,6 +2288,7 @@ Apart from these, the following properties are also available, and may be useful + @@ -2188,6 +2303,7 @@ Apart from these, the following properties are also available, and may be useful the conf values of spark.executor.cores and spark.task.cpus minimum 1. Default unit is bytes, unless otherwise specified. + @@ -2211,6 +2327,7 @@ Apart from these, the following properties are also available, and may be useful in order to assign resource slots (e.g. a 0.2222 configuration, or 1/0.2222 slots will become 4 tasks/resource, not 5). + @@ -2275,13 +2392,14 @@ Apart from these, the following properties are also available, and may be useful +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.cores.max (not set)0.8.0
    spark.locality.wait.process0.8.0
    spark.locality.wait.rack Customize the locality wait for rack locality. 0.8.0
    spark.scheduler.maxRegisteredResourcesWaitingTime Maximum amount of time to wait for resources to register before scheduling begins. 1.1.1
    spark.scheduler.minRegisteredResourcesRatio1.1.1
    spark.scheduler.mode0.8.0
    spark.scheduler.revive.interval The interval length for the scheduler to revive the worker resource offers to run tasks. 0.8.1
    spark.scheduler.listenerbus.eventqueue.capacity3.0.0
    spark.scheduler.listenerbus.eventqueue.appStatus.capacity3.0.0
    spark.scheduler.listenerbus.eventqueue.executorManagement.capacity3.0.0
    spark.scheduler.listenerbus.eventqueue.eventLog.capacity3.0.0
    spark.scheduler.listenerbus.eventqueue.streams.capacity3.0.0
    spark.scheduler.resource.profileMergeConflictsfalse + If set to "true", Spark will merge ResourceProfiles when different profiles are specified + in RDDs that get combined into a single stage. When they are merged, Spark chooses the maximum of + each resource and creates a new ResourceProfile. The default of false results in Spark throwing + an exception if multiple different ResourceProfiles are found in RDDs going into the same stage. + 3.1.0
    spark.scheduler.blacklist.unschedulableTaskSetTimeout2.4.1
    spark.blacklist.enabled0.6.0
    spark.speculation.interval How often Spark will check for tasks to speculate. 0.6.0
    spark.speculation.multiplier How many times slower a task is than the median to be considered for speculation. 0.6.0
    spark.speculation.quantile Fraction of tasks which must be complete before speculation is enabled for a particular stage. 0.6.0
    spark.speculation.task.duration.threshold3.0.0
    spark.task.cpus3.0.0
    spark.task.maxFailures Number of consecutive stage attempts allowed before a stage is aborted. 2.2.0
    ### Barrier Execution Mode - + @@ -2291,6 +2409,7 @@ Apart from these, the following properties are also available, and may be useful configured time, throw a SparkException to fail all the tasks. The default value is set to 31536000(3600 * 24 * 365) so the barrier() call shall wait for one year. + @@ -2305,6 +2424,7 @@ Apart from these, the following properties are also available, and may be useful config only applies to jobs that contain one or more barrier stages, we won't perform the check on non-barrier jobs. + @@ -2319,13 +2439,14 @@ Apart from these, the following properties are also available, and may be useful applies to jobs that contain one or more barrier stages, we won't perform the check on non-barrier jobs. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.barrier.sync.timeout 365d2.4.0
    spark.scheduler.barrier.maxConcurrentTasksCheck.interval2.4.0
    spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures2.4.0
    ### Dynamic Allocation - + @@ -2335,7 +2456,8 @@ Apart from these, the following properties are also available, and may be useful For more detail, see the description here.

    - This requires spark.shuffle.service.enabled to be set. + This requires spark.shuffle.service.enabled or + spark.dynamicAllocation.shuffleTracking.enabled to be set. The following configurations are also relevant: spark.dynamicAllocation.minExecutors, spark.dynamicAllocation.maxExecutors, and @@ -2463,13 +2585,14 @@ like shuffle, just replace "rpc" with "shuffle" in the property names except spark.{driver|executor}.rpc.netty.dispatcher.numThreads, which is only for RPC module.
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.dynamicAllocation.enabled false
    - + + @@ -2477,6 +2600,7 @@ like shuffle, just replace "rpc" with "shuffle" in the property names except Fall back on spark.rpc.io.clientThreads + @@ -2484,6 +2608,7 @@ like shuffle, just replace "rpc" with "shuffle" in the property names except Fall back on spark.rpc.netty.dispatcher.numThreads +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.{driver|executor}.rpc.io.serverThreads Fall back on spark.rpc.io.serverThreads Number of threads used in the server thread pool1.6.0
    spark.{driver|executor}.rpc.io.clientThreads Number of threads used in the client thread pool1.6.0
    spark.{driver|executor}.rpc.netty.dispatcher.numThreads Number of threads used in RPC message dispatcher thread pool3.0.0
    @@ -2510,7 +2635,7 @@ Spark subsystems. ### Spark Streaming - + @@ -2523,6 +2648,7 @@ Spark subsystems. spark.streaming.receiver.maxRate and spark.streaming.kafka.maxRatePerPartition if they are set (see below). + @@ -2531,6 +2657,7 @@ Spark subsystems. This is the initial maximum receiving rate at which each receiver will receive data for the first batch when the backpressure mechanism is enabled. + @@ -2541,6 +2668,7 @@ Spark subsystems. performance tuning section in the Spark Streaming programming guide for more details. + @@ -2552,6 +2680,7 @@ Spark subsystems. See the deployment guide in the Spark Streaming programming guide for mode details. + @@ -2562,6 +2691,7 @@ Spark subsystems. See the deployment guide in the Spark Streaming programming guide for more details. + @@ -2573,6 +2703,7 @@ Spark subsystems. streaming application as they will not be cleared automatically. But it comes at the cost of higher memory usage in Spark. + @@ -2581,6 +2712,7 @@ Spark subsystems. If true, Spark shuts down the StreamingContext gracefully on JVM shutdown rather than immediately. + @@ -2591,14 +2723,16 @@ Spark subsystems. Kafka Integration guide for more details. + - - - + + + + @@ -2606,6 +2740,7 @@ Spark subsystems. + @@ -2615,6 +2750,7 @@ Spark subsystems. when you want to use S3 (or any file system that does not support flushing) for the metadata WAL on the driver. + @@ -2624,6 +2760,7 @@ Spark subsystems. when you want to use S3 (or any file system that does not support flushing) for the data WAL on the receivers. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.streaming.backpressure.enabled false1.5.0
    spark.streaming.backpressure.initialRate2.0.0
    spark.streaming.blockInterval0.8.0
    spark.streaming.receiver.maxRate1.0.2
    spark.streaming.receiver.writeAheadLog.enable1.2.1
    spark.streaming.unpersist0.9.0
    spark.streaming.stopGracefullyOnShutdown1.4.0
    spark.streaming.kafka.maxRatePerPartition1.3.0
    spark.streaming.kafka.minRatePerPartition1 - Minimum rate (number of records per second) at which data will be read from each Kafka - partition when using the new Kafka direct stream API. - spark.streaming.kafka.minRatePerPartition1 + Minimum rate (number of records per second) at which data will be read from each Kafka + partition when using the new Kafka direct stream API. + 2.4.0
    spark.streaming.ui.retainedBatches How many batches the Spark Streaming UI and status APIs remember before garbage collecting. 1.0.0
    spark.streaming.driver.writeAheadLog.closeFileAfterWrite1.6.0
    spark.streaming.receiver.writeAheadLog.closeFileAfterWrite1.6.0
    @@ -2653,7 +2790,7 @@ Spark subsystems. Executable for executing R scripts in client modes for driver. Ignored in cluster modes. - + 1.5.3 spark.r.shell.command @@ -2662,7 +2799,7 @@ Spark subsystems. Executable for executing sparkR shell in client modes for driver. Ignored in cluster modes. It is the same as environment variable SPARKR_DRIVER_R, but take precedence over it. spark.r.shell.command is used for sparkR shell while spark.r.driver.command is used for running R script. - + 2.1.0 spark.r.backendConnectionTimeout @@ -2686,7 +2823,7 @@ Spark subsystems. ### GraphX - + @@ -2694,6 +2831,7 @@ Spark subsystems. Checkpoint interval for graph and message in Pregel. It used to avoid stackOverflowError due to long lineage chains after lots of iterations. The checkpoint is disabled by default. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.graphx.pregel.checkpointInterval -12.2.0
    diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index fdb0afad6af9b..66a489bcc8be3 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -26,7 +26,7 @@ license: | - The `org.apache.spark.ExecutorPlugin` interface and related configuration has been replaced with `org.apache.spark.plugin.SparkPlugin`, which adds new functionality. Plugins using the old - interface need to be modified to extend the new interfaces. Check the + interface must be modified to extend the new interfaces. Check the [Monitoring](monitoring.html) guide for more details. - Deprecated method `TaskContext.isRunningLocally` has been removed. Local execution was removed and it always has returned `false`. @@ -35,6 +35,6 @@ license: | - Deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. -- Event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. +- Event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark wrote the event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. -- A new protocol for fetching shuffle blocks is used. It's recommended that external shuffle services be upgraded when running Spark 3.0 apps. Old external shuffle services can still be used by setting the configuration `spark.shuffle.useOldFetchProtocol` to `true`. Otherwise, Spark may run into errors with messages like `IllegalArgumentException: Unexpected message type: `. +- A new protocol for fetching shuffle blocks is used. It's recommended that external shuffle services be upgraded when running Spark 3.0 apps. You can still use old external shuffle services by setting the configuration `spark.shuffle.useOldFetchProtocol` to `true`. Otherwise, Spark may run into errors with messages like `IllegalArgumentException: Unexpected message type: `. diff --git a/docs/css/main.css b/docs/css/main.css index dc05d287996be..bb34d6e9bec0d 100755 --- a/docs/css/main.css +++ b/docs/css/main.css @@ -2,6 +2,37 @@ Author's custom styles ========================================================================== */ +table { + margin: 15px 0; + padding: 0; +} + +table tr { + border-top: 1px solid #cccccc; + background-color: white; + margin: 0; + padding: 0; +} + +table tr:nth-child(2n) { + background-color: #F1F4F5; +} + +table tr th { + font-weight: bold; + border: 1px solid #cccccc; + text-align: left; + margin: 0; + padding: 6px 13px; +} + +table tr td { + border: 1px solid #cccccc; + text-align: left; + margin: 0; + padding: 6px 13px; +} + .navbar .brand { height: 50px; width: 110px; diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index 9d5388005e587..ce0831d3fb1a2 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -9,9 +9,9 @@ license: | 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. @@ -59,11 +59,11 @@ parameter to select between these two algorithms, or leave it unset and Spark wi ### Binomial logistic regression -For more background and more details about the implementation of binomial logistic regression, refer to the documentation of [logistic regression in `spark.mllib`](mllib-linear-methods.html#logistic-regression). +For more background and more details about the implementation of binomial logistic regression, refer to the documentation of [logistic regression in `spark.mllib`](mllib-linear-methods.html#logistic-regression). **Examples** -The following example shows how to train binomial and multinomial logistic regression +The following example shows how to train binomial and multinomial logistic regression models for binary classification with elastic net regularization. `elasticNetParam` corresponds to $\alpha$ and `regParam` corresponds to $\lambda$. @@ -156,7 +156,7 @@ classes and $J$ is the number of features. If the algorithm is fit with an inter intercepts is available. > Multinomial coefficients are available as `coefficientMatrix` and intercepts are available as `interceptVector`. - + > `coefficients` and `intercept` methods on a logistic regression model trained with multinomial family are not supported. Use `coefficientMatrix` and `interceptVector` instead. The conditional probabilities of the outcome classes $k \in \{1, 2, ..., K\}$ are modeled using the softmax function. @@ -175,7 +175,7 @@ For a detailed derivation please see [here](https://en.wikipedia.org/wiki/Multin **Examples** -The following example shows how to train a multiclass logistic regression +The following example shows how to train a multiclass logistic regression model with elastic net regularization, as well as extract the multiclass training summary for evaluating the model. @@ -291,7 +291,7 @@ Refer to the [R API docs](api/R/spark.randomForest.html) for more details. ## Gradient-boosted tree classifier -Gradient-boosted trees (GBTs) are a popular classification and regression method using ensembles of decision trees. +Gradient-boosted trees (GBTs) are a popular classification and regression method using ensembles of decision trees. More information about the `spark.ml` implementation can be found further in the [section on GBTs](#gradient-boosted-trees-gbts). **Examples** @@ -332,10 +332,10 @@ Refer to the [R API docs](api/R/spark.gbt.html) for more details. ## Multilayer perceptron classifier -Multilayer perceptron classifier (MLPC) is a classifier based on the [feedforward artificial neural network](https://en.wikipedia.org/wiki/Feedforward_neural_network). -MLPC consists of multiple layers of nodes. -Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes map inputs to outputs -by a linear combination of the inputs with the node's weights `$\wv$` and bias `$\bv$` and applying an activation function. +Multilayer perceptron classifier (MLPC) is a classifier based on the [feedforward artificial neural network](https://en.wikipedia.org/wiki/Feedforward_neural_network). +MLPC consists of multiple layers of nodes. +Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes map inputs to outputs +by a linear combination of the inputs with the node's weights `$\wv$` and bias `$\bv$` and applying an activation function. This can be written in matrix form for MLPC with `$K+1$` layers as follows: `\[ \mathrm{y}(\x) = \mathrm{f_K}(...\mathrm{f_2}(\wv_2^T\mathrm{f_1}(\wv_1^T \x+b_1)+b_2)...+b_K) @@ -348,7 +348,7 @@ Nodes in the output layer use softmax function: `\[ \mathrm{f}(z_i) = \frac{e^{z_i}}{\sum_{k=1}^N e^{z_k}} \]` -The number of nodes `$N$` in the output layer corresponds to the number of classes. +The number of nodes `$N$` in the output layer corresponds to the number of classes. MLPC employs backpropagation for learning the model. We use the logistic loss function for optimization and L-BFGS as an optimization routine. @@ -393,7 +393,7 @@ or set of hyperplanes in a high- or infinite-dimensional space, which can be use regression, or other tasks. Intuitively, a good separation is achieved by the hyperplane that has the largest distance to the nearest training-data points of any class (so-called functional margin), since in general the larger the margin the lower the generalization error of the classifier. LinearSVC -in Spark ML supports binary classification with linear SVM. Internally, it optimizes the +in Spark ML supports binary classification with linear SVM. Internally, it optimizes the [Hinge Loss](https://en.wikipedia.org/wiki/Hinge_loss) using OWLQN optimizer. @@ -469,8 +469,8 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classificat ## Naive Bayes -[Naive Bayes classifiers](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) are a family of simple -probabilistic, multiclass classifiers based on applying Bayes' theorem with strong (naive) independence +[Naive Bayes classifiers](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) are a family of simple +probabilistic, multiclass classifiers based on applying Bayes' theorem with strong (naive) independence assumptions between every pair of features. Naive Bayes can be trained very efficiently. With a single pass over the training data, @@ -494,7 +494,7 @@ For document classification, the input feature vectors should usually be sparse Since the training data is only used once, it is not necessary to cache it. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by -setting the parameter $\lambda$ (default to $1.0$). +setting the parameter $\lambda$ (default to $1.0$). **Examples** @@ -563,6 +563,15 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classificat {% include_example python/ml/fm_classifier_example.py %} +
    + +Refer to the [R API docs](api/R/spark.fmClassifier.html) for more details. + +Note: At the moment SparkR doesn't suport feature scaling. + +{% include_example r/ml/fmClassifier.R %} +
    + @@ -605,6 +614,13 @@ More details on parameters can be found in the [Python API documentation](api/py {% include_example python/ml/linear_regression_with_elastic_net.py %} +
    + +More details on parameters can be found in the [R API documentation](api/R/spark.lm.html). + +{% include_example r/ml/lm_with_elastic_net.R %} +
    + ## Generalized linear regression @@ -620,7 +636,7 @@ Currently in `spark.ml`, only a subset of the exponential family distributions a **NOTE**: Spark currently only supports up to 4096 features through its `GeneralizedLinearRegression` interface, and will throw an exception if this constraint is exceeded. See the [advanced section](ml-advanced) for more details. - Still, for linear and logistic regression, models with an increased number of features can be trained + Still, for linear and logistic regression, models with an increased number of features can be trained using the `LinearRegression` and `LogisticRegression` estimators. GLMs require exponential family distributions that can be written in their "canonical" or "natural" form, aka @@ -840,7 +856,7 @@ Refer to the [R API docs](api/R/spark.randomForest.html) for more details. ## Gradient-boosted tree regression -Gradient-boosted trees (GBTs) are a popular regression method using ensembles of decision trees. +Gradient-boosted trees (GBTs) are a popular regression method using ensembles of decision trees. More information about the `spark.ml` implementation can be found further in the [section on GBTs](#gradient-boosted-trees-gbts). **Examples** @@ -883,16 +899,16 @@ Refer to the [R API docs](api/R/spark.gbt.html) for more details. ## Survival regression -In `spark.ml`, we implement the [Accelerated failure time (AFT)](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) -model which is a parametric survival regression model for censored data. -It describes a model for the log of survival time, so it's often called a +In `spark.ml`, we implement the [Accelerated failure time (AFT)](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) +model which is a parametric survival regression model for censored data. +It describes a model for the log of survival time, so it's often called a log-linear model for survival analysis. Different from a [Proportional hazards](https://en.wikipedia.org/wiki/Proportional_hazards_model) model -designed for the same purpose, the AFT model is easier to parallelize +designed for the same purpose, the AFT model is easier to parallelize because each instance contributes to the objective function independently. -Given the values of the covariates $x^{'}$, for random lifetime $t_{i}$ of -subjects i = 1, ..., n, with possible right-censoring, +Given the values of the covariates $x^{'}$, for random lifetime $t_{i}$ of +subjects i = 1, ..., n, with possible right-censoring, the likelihood function under the AFT model is given as: `\[ L(\beta,\sigma)=\prod_{i=1}^n[\frac{1}{\sigma}f_{0}(\frac{\log{t_{i}}-x^{'}\beta}{\sigma})]^{\delta_{i}}S_{0}(\frac{\log{t_{i}}-x^{'}\beta}{\sigma})^{1-\delta_{i}} @@ -906,8 +922,8 @@ assumes the form: Where $S_{0}(\epsilon_{i})$ is the baseline survivor function, and $f_{0}(\epsilon_{i})$ is the corresponding density function. -The most commonly used AFT model is based on the Weibull distribution of the survival time. -The Weibull distribution for lifetime corresponds to the extreme value distribution for the +The most commonly used AFT model is based on the Weibull distribution of the survival time. +The Weibull distribution for lifetime corresponds to the extreme value distribution for the log of the lifetime, and the $S_{0}(\epsilon)$ function is: `\[ S_{0}(\epsilon_{i})=\exp(-e^{\epsilon_{i}}) @@ -926,15 +942,15 @@ The gradient functions for $\beta$ and $\log\sigma$ respectively are: `\[ \frac{\partial (-\iota)}{\partial \beta}=\sum_{1=1}^{n}[\delta_{i}-e^{\epsilon_{i}}]\frac{x_{i}}{\sigma} \]` -`\[ +`\[ \frac{\partial (-\iota)}{\partial (\log\sigma)}=\sum_{i=1}^{n}[\delta_{i}+(\delta_{i}-e^{\epsilon_{i}})\epsilon_{i}] \]` -The AFT model can be formulated as a convex optimization problem, -i.e. the task of finding a minimizer of a convex function $-\iota(\beta,\sigma)$ +The AFT model can be formulated as a convex optimization problem, +i.e. the task of finding a minimizer of a convex function $-\iota(\beta,\sigma)$ that depends on the coefficients vector $\beta$ and the log of scale parameter $\log\sigma$. The optimization algorithm underlying the implementation is L-BFGS. -The implementation matches the result from R's survival function +The implementation matches the result from R's survival function [survreg](https://stat.ethz.ch/R-manual/R-devel/library/survival/html/survreg.html) > When fitting AFTSurvivalRegressionModel without intercept on dataset with constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero columns. This behavior is different from R survival::survreg. @@ -1085,6 +1101,15 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression. {% include_example python/ml/fm_regressor_example.py %} +
    + +Refer to the [R API documentation](api/R/spark.fmRegressor.html) for more details. + +Note: At the moment SparkR doesn't suport feature scaling. + +{% include_example r/ml/fmRegressor.R %} +
    + @@ -1174,7 +1199,7 @@ The main differences between this API and the [original MLlib Decision Tree API] The Pipelines API for Decision Trees offers a bit more functionality than the original API. -In particular, for classification, users can get the predicted probability of each class (a.k.a. class conditional probabilities); +In particular, for classification, users can get the predicted probability of each class (a.k.a. class conditional probabilities); for regression, users can get the biased sample variance of prediction. Ensembles of trees (Random Forests and Gradient-Boosted Trees) are described below in the [Tree ensembles section](#tree-ensembles). @@ -1420,4 +1445,3 @@ Note that `GBTClassifier` currently only supports binary labels. In the future, `GBTClassifier` will also output columns for `rawPrediction` and `probability`, just as `RandomForestClassifier` does. - diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 2037285f3f12c..ddce98b32f941 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -78,7 +78,7 @@ The most popular native BLAS such as [Intel MKL](https://software.intel.com/en-u Configuring these BLAS implementations to use a single thread for operations may actually improve performance (see [SPARK-21305](https://issues.apache.org/jira/browse/SPARK-21305)). It is usually optimal to match this to the number of cores each Spark task is configured to use, which is 1 by default and typically left at 1. -Please refer to resources like the following to understand how to configure the number of threads these BLAS implementations use: [Intel MKL](https://software.intel.com/en-us/articles/recommended-settings-for-calling-intel-mkl-routines-from-multi-threaded-applications) and [OpenBLAS](https://github.com/xianyi/OpenBLAS/wiki/faq#multi-threaded). +Please refer to resources like the following to understand how to configure the number of threads these BLAS implementations use: [Intel MKL](https://software.intel.com/en-us/articles/recommended-settings-for-calling-intel-mkl-routines-from-multi-threaded-applications) or [Intel oneMKL](https://software.intel.com/en-us/onemkl-linux-developer-guide-improving-performance-with-threading) and [OpenBLAS](https://github.com/xianyi/OpenBLAS/wiki/faq#multi-threaded). Note that if nativeBLAS is not properly configured in system, java implementation(f2jBLAS) will be used as fallback option. To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. @@ -91,10 +91,10 @@ The list below highlights some of the new features and enhancements added to MLl release of Spark: * Multiple columns support was added to `Binarizer` ([SPARK-23578](https://issues.apache.org/jira/browse/SPARK-23578)), `StringIndexer` ([SPARK-11215](https://issues.apache.org/jira/browse/SPARK-11215)), `StopWordsRemover` ([SPARK-29808](https://issues.apache.org/jira/browse/SPARK-29808)) and PySpark `QuantileDiscretizer` ([SPARK-22796](https://issues.apache.org/jira/browse/SPARK-22796)). -* Support Tree-Based Feature Transformation was added +* Tree-Based Feature Transformation was added ([SPARK-13677](https://issues.apache.org/jira/browse/SPARK-13677)). * Two new evaluators `MultilabelClassificationEvaluator` ([SPARK-16692](https://issues.apache.org/jira/browse/SPARK-16692)) and `RankingEvaluator` ([SPARK-28045](https://issues.apache.org/jira/browse/SPARK-28045)) were added. -* Sample weights support was added in `DecisionTreeClassifier/Regressor` ([SPARK-19591](https://issues.apache.org/jira/browse/SPARK-19591)), `RandomForestClassifier/Regressor` ([SPARK-9478](https://issues.apache.org/jira/browse/SPARK-9478)), `GBTClassifier/Regressor` ([SPARK-9612](https://issues.apache.org/jira/browse/SPARK-9612)), `RegressionEvaluator` ([SPARK-24102](https://issues.apache.org/jira/browse/SPARK-24102)), `BinaryClassificationEvaluator` ([SPARK-24103](https://issues.apache.org/jira/browse/SPARK-24103)), `BisectingKMeans` ([SPARK-30351](https://issues.apache.org/jira/browse/SPARK-30351)), `KMeans` ([SPARK-29967](https://issues.apache.org/jira/browse/SPARK-29967)) and `GaussianMixture` ([SPARK-30102](https://issues.apache.org/jira/browse/SPARK-30102)). +* Sample weights support was added in `DecisionTreeClassifier/Regressor` ([SPARK-19591](https://issues.apache.org/jira/browse/SPARK-19591)), `RandomForestClassifier/Regressor` ([SPARK-9478](https://issues.apache.org/jira/browse/SPARK-9478)), `GBTClassifier/Regressor` ([SPARK-9612](https://issues.apache.org/jira/browse/SPARK-9612)), `MulticlassClassificationEvaluator` ([SPARK-24101](https://issues.apache.org/jira/browse/SPARK-24101)), `RegressionEvaluator` ([SPARK-24102](https://issues.apache.org/jira/browse/SPARK-24102)), `BinaryClassificationEvaluator` ([SPARK-24103](https://issues.apache.org/jira/browse/SPARK-24103)), `BisectingKMeans` ([SPARK-30351](https://issues.apache.org/jira/browse/SPARK-30351)), `KMeans` ([SPARK-29967](https://issues.apache.org/jira/browse/SPARK-29967)) and `GaussianMixture` ([SPARK-30102](https://issues.apache.org/jira/browse/SPARK-30102)). * R API for `PowerIterationClustering` was added ([SPARK-19827](https://issues.apache.org/jira/browse/SPARK-19827)). * Added Spark ML listener for tracking ML pipeline status @@ -105,10 +105,10 @@ release of Spark: ([SPARK-28399](https://issues.apache.org/jira/browse/SPARK-28399)). * [`Factorization Machines`](ml-classification-regression.html#factorization-machines) classifier and regressor were added ([SPARK-29224](https://issues.apache.org/jira/browse/SPARK-29224)). -* Gaussian Naive Bayes ([SPARK-16872](https://issues.apache.org/jira/browse/SPARK-16872)) and Complement Naive Bayes ([SPARK-29942](https://issues.apache.org/jira/browse/SPARK-29942)) were added. +* Gaussian Naive Bayes Classifier ([SPARK-16872](https://issues.apache.org/jira/browse/SPARK-16872)) and Complement Naive Bayes Classifier ([SPARK-29942](https://issues.apache.org/jira/browse/SPARK-29942)) were added. * ML function parity between Scala and Python ([SPARK-28958](https://issues.apache.org/jira/browse/SPARK-28958)). -* `predictRaw` is made public in all the Classification models. `predictProbability` is made public in all the Classification models except `LinearSVCModel`. +* `predictRaw` is made public in all the Classification models. `predictProbability` is made public in all the Classification models except `LinearSVCModel` ([SPARK-30358](https://issues.apache.org/jira/browse/SPARK-30358)). # Migration Guide diff --git a/docs/ml-tuning.md b/docs/ml-tuning.md index 49e23684e5974..274f1951011bd 100644 --- a/docs/ml-tuning.md +++ b/docs/ml-tuning.md @@ -65,9 +65,11 @@ At a high level, these model selection tools work as follows: The `Evaluator` can be a [`RegressionEvaluator`](api/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.html) for regression problems, a [`BinaryClassificationEvaluator`](api/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.html) -for binary data, or a [`MulticlassClassificationEvaluator`](api/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.html) -for multiclass problems. The default metric used to choose the best `ParamMap` can be overridden by the `setMetricName` -method in each of these evaluators. +for binary data, a [`MulticlassClassificationEvaluator`](api/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.html) +for multiclass problems, a [`MultilabelClassificationEvaluator`](api/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.html) + for multi-label classifications, or a +[`RankingEvaluator`](api/scala/org/apache/spark/ml/evaluation/RankingEvaluator.html) for ranking problems. The default metric used to +choose the best `ParamMap` can be overridden by the `setMetricName` method in each of these evaluators. To help construct the parameter grid, users can use the [`ParamGridBuilder`](api/scala/org/apache/spark/ml/tuning/ParamGridBuilder.html) utility. By default, sets of parameters from the parameter grid are evaluated in serial. Parameter evaluation can be done in parallel by setting `parallelism` with a value of 2 or more (a value of 1 will be serial) before running model selection with `CrossValidator` or `TrainValidationSplit`. diff --git a/docs/monitoring.md b/docs/monitoring.md index 0b9ddc7db42bf..b72fd027da597 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -713,31 +713,75 @@ A list of the available metrics, with a short description: ### Executor Metrics Executor-level metrics are sent from each executor to the driver as part of the Heartbeat to describe the performance metrics of Executor itself like JVM heap memory, GC information. -Executor metric values and their measured peak values per executor are exposed via the REST API at the end point `/applications/[app-id]/executors`. -In addition, aggregated per-stage peak values of the executor metrics are written to the event log if `spark.eventLog.logStageExecutorMetrics` is true. -Executor metrics are also exposed via the Spark metrics system based on the Dropwizard metrics library. +Executor metric values and their measured memory peak values per executor are exposed via the REST API in JSON format and in Prometheus format. +The JSON end point is exposed at: `/applications/[app-id]/executors`, and the Prometheus endpoint at: `/metrics/executors/prometheus`. +The Prometheus endpoint is conditional to a configuration parameter: `spark.ui.prometheus.enabled=true` (the default is `false`). +In addition, aggregated per-stage peak values of the executor memory metrics are written to the event log if +`spark.eventLog.logStageExecutorMetrics` is true. +Executor memory metrics are also exposed via the Spark metrics system based on the Dropwizard metrics library. A list of the available metrics, with a short description: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - - + - + - + diff --git a/docs/pyspark-migration-guide.md b/docs/pyspark-migration-guide.md index 03e062f002680..92388fffcea52 100644 --- a/docs/pyspark-migration-guide.md +++ b/docs/pyspark-migration-guide.md @@ -27,67 +27,25 @@ Many items of SQL migration can be applied when migrating PySpark to higher vers Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). ## Upgrading from PySpark 2.4 to 3.0 +- In Spark 3.0, PySpark requires a pandas version of 0.23.2 or higher to use pandas related functionality, such as `toPandas`, `createDataFrame` from pandas DataFrame, and so on. - - Since Spark 3.0, PySpark requires a Pandas version of 0.23.2 or higher to use Pandas related functionality, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. - - - Since Spark 3.0, PySpark requires a PyArrow version of 0.12.1 or higher to use PyArrow related functionality, such as `pandas_udf`, `toPandas` and `createDataFrame` with "spark.sql.execution.arrow.enabled=true", etc. - - - In PySpark, when creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 3.0, the builder comes to not update the configurations. This is the same behavior as Java/Scala API in 2.3 and above. If you want to update them, you need to update them prior to creating a `SparkSession`. - - - In PySpark, when Arrow optimization is enabled, if Arrow version is higher than 0.11.0, Arrow can perform safe type conversion when converting Pandas.Series to Arrow array during serialization. Arrow will raise errors when detecting unsafe type conversion like overflow. Setting `spark.sql.execution.pandas.convertToArrowArraySafely` to true can enable it. The default setting is false. PySpark's behavior for Arrow versions is illustrated in the table below: -
    Executor Level Metric name Short description
    rddBlocksRDD blocks in the block manager of this executor.
    memoryUsedStorage memory used by this executor.
    diskUsedDisk space used for RDD storage by this executor.
    totalCoresNumber of cores available in this executor.
    maxTasksMaximum number of tasks that can run concurrently in this executor.
    activeTasksNumber of tasks currently executing.
    failedTasksNumber of tasks that have failed in this executor.
    completedTasksNumber of tasks that have completed in this executor.
    totalTasksTotal number of tasks (running, failed and completed) in this executor.
    totalDurationElapsed time the JVM spent executing tasks in this executor. + The value is expressed in milliseconds.
    totalGCTimeElapsed time the JVM spent in garbage collection summed in this Executor. + Elapsed time the JVM spent in garbage collection summed in this executor. The value is expressed in milliseconds.
    totalInputBytesTotal input bytes summed in this Executor.Total input bytes summed in this executor.
    totalShuffleReadTotal shuffer read bytes summed in this Executor.Total shuffle read bytes summed in this executor.
    totalShuffleWriteTotal shuffer write bytes summed in this Executor.Total shuffle write bytes summed in this executor.
    maxMemory
    - - - - - - - - - - - - - - - - - - - - -
    - PyArrow version - - Integer Overflow - - Floating Point Truncation -
    - version < 0.11.0 - - Raise error - - Silently allows -
    - version > 0.11.0, arrowSafeTypeConversion=false - - Silent overflow - - Silently allows -
    - version > 0.11.0, arrowSafeTypeConversion=true - - Raise error - - Raise error -
    - - - Since Spark 3.0, `createDataFrame(..., verifySchema=True)` validates `LongType` as well in PySpark. Previously, `LongType` was not verified and resulted in `None` in case the value overflows. To restore this behavior, `verifySchema` can be set to `False` to disable the validation. - - - Since Spark 3.0, `Column.getItem` is fixed such that it does not call `Column.apply`. Consequently, if `Column` is used as an argument to `getItem`, the indexing operator should be used. - For example, `map_col.getItem(col('id'))` should be replaced with `map_col[col('id')]`. - - - As of Spark 3.0 `Row` field names are no longer sorted alphabetically when constructing with named arguments for Python versions 3.6 and above, and the order of fields will match that as entered. To enable sorted fields by default, as in Spark 2.4, set the environment variable `PYSPARK_ROW_FIELD_SORTING_ENABLED` to "true" for both executors and driver - this environment variable must be consistent on all executors and driver; otherwise, it may cause failures or incorrect answers. For Python versions less than 3.6, the field names will be sorted alphabetically as the only option. +- In Spark 3.0, PySpark requires a PyArrow version of 0.12.1 or higher to use PyArrow related functionality, such as `pandas_udf`, `toPandas` and `createDataFrame` with "spark.sql.execution.arrow.enabled=true", etc. + +- In PySpark, when creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. In 3.0, the builder comes to not update the configurations. This is the same behavior as Java/Scala API in 2.3 and above. If you want to update them, you need to update them prior to creating a `SparkSession`. + +- In PySpark, when Arrow optimization is enabled, if Arrow version is higher than 0.11.0, Arrow can perform safe type conversion when converting `pandas.Series` to an Arrow array during serialization. Arrow raises errors when detecting unsafe type conversions like overflow. You enable it by setting `spark.sql.execution.pandas.convertToArrowArraySafely` to `true`. The default setting is `false`. PySpark behavior for Arrow versions is illustrated in the following table: + + | PyArrow version | Integer overflow | Floating point truncation | + | ---------------- | ---------------- | ------------------------- | + | 0.11.0 and below | Raise error | Silently allows | + | \> 0.11.0, arrowSafeTypeConversion=false | Silent overflow | Silently allows | + | \> 0.11.0, arrowSafeTypeConversion=true | Raise error | Raise error | + +- In Spark 3.0, `createDataFrame(..., verifySchema=True)` validates `LongType` as well in PySpark. Previously, `LongType` was not verified and resulted in `None` in case the value overflows. To restore this behavior, `verifySchema` can be set to `False` to disable the validation. + +- In Spark 3.0, `Column.getItem` is fixed such that it does not call `Column.apply`. Consequently, if `Column` is used as an argument to `getItem`, the indexing operator should be used. For example, `map_col.getItem(col('id'))` should be replaced with `map_col[col('id')]`. + +- As of Spark 3.0, `Row` field names are no longer sorted alphabetically when constructing with named arguments for Python versions 3.6 and above, and the order of fields will match that as entered. To enable sorted fields by default, as in Spark 2.4, set the environment variable `PYSPARK_ROW_FIELD_SORTING_ENABLED` to `true` for both executors and driver - this environment variable must be consistent on all executors and driver; otherwise, it may cause failures or incorrect answers. For Python versions less than 3.6, the field names will be sorted alphabetically as the only option. ## Upgrading from PySpark 2.3 to 2.4 diff --git a/docs/security.md b/docs/security.md index a4556e242cffc..aef6e69382a80 100644 --- a/docs/security.md +++ b/docs/security.md @@ -66,11 +66,12 @@ also see their authentication secret. Access control rules should be properly se Kubernetes admin to ensure that Spark authentication is secure. - + + @@ -78,6 +79,7 @@ Kubernetes admin to ensure that Spark authentication is secure. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.authenticate false Whether Spark authenticates its internal connections.1.0.0
    spark.authenticate.secret The secret key used authentication. See above for when this configuration should be set. 1.0.0
    @@ -85,7 +87,7 @@ Alternatively, one can mount authentication secrets using files and Kubernetes s the user mounts into their pods. - + @@ -94,6 +96,7 @@ the user mounts into their pods. contents of the file have been securely generated. This file is loaded on both the driver and the executors unless other settings override this (see below). + @@ -106,6 +109,7 @@ the user mounts into their pods. and the executors can both use files to load the secret key. Ensure that the contents of the file on the driver is identical to the contents of the file on the executors. + @@ -118,6 +122,7 @@ the user mounts into their pods. and the executors can both use files to load the secret key. Ensure that the contents of the file on the driver is identical to the contents of the file on the executors. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.authenticate.secret.file None3.0.0
    spark.authenticate.secret.driver.file3.0.0
    spark.authenticate.secret.executor.file3.0.0
    @@ -138,13 +143,14 @@ is still required when talking to shuffle services from Spark versions older tha The following table describes the different options available for configuring this feature. - + + @@ -152,6 +158,7 @@ The following table describes the different options available for configuring th + @@ -160,6 +167,7 @@ The following table describes the different options available for configuring th 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. + @@ -169,6 +177,7 @@ The following table describes the different options available for configuring th use. The config name should be the name of commons-crypto configuration without the commons.crypto prefix. + @@ -179,6 +188,7 @@ The following table describes the different options available for configuring th do not support the internal Spark authentication protocol. On the shuffle service side, disabling this feature will block older clients from authenticating. + @@ -186,6 +196,7 @@ The following table describes the different options available for configuring th + @@ -194,6 +205,7 @@ The following table describes the different options available for configuring th Disable unencrypted connections for ports using SASL authentication. This will deny connections from clients that have authentication enabled, but do not request SASL-based encryption. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.network.crypto.enabled false Enable AES-based RPC encryption, including the new authentication protocol added in 2.2.0. 2.2.0
    spark.network.crypto.keyLength The length in bits of the encryption key to generate. Valid values are 128, 192 and 256. 2.2.0
    spark.network.crypto.keyFactoryAlgorithm2.2.0
    spark.network.crypto.config.*2.2.0
    spark.network.crypto.saslFallback2.2.0
    spark.authenticate.enableSaslEncryption Enable SASL-based encrypted communication. 2.2.0
    spark.network.sasl.serverAlwaysEncrypt1.4.0
    @@ -208,7 +220,7 @@ encrypting output data generated by applications with APIs such as `saveAsHadoop The following settings cover enabling encryption for data written to disk: - + @@ -216,6 +228,7 @@ The following settings cover enabling encryption for data written to disk: Enable local disk I/O encryption. Currently supported by all modes except Mesos. It's strongly recommended that RPC encryption be enabled when using this feature. + @@ -223,6 +236,7 @@ The following settings cover enabling encryption for data written to disk: + @@ -232,6 +246,7 @@ The following settings cover enabling encryption for data written to disk: described in the KeyGenerator section of the Java Cryptography Architecture Standard Algorithm Name Documentation. + @@ -241,6 +256,7 @@ The following settings cover enabling encryption for data written to disk: use. The config name should be the name of commons-crypto configuration without the commons.crypto prefix. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.io.encryption.enabled false2.1.0
    spark.io.encryption.keySizeBits IO encryption key size in bits. Supported values are 128, 192 and 256. 2.1.0
    spark.io.encryption.keygen.algorithm2.1.0
    spark.io.encryption.commons.config.*2.1.0
    @@ -272,7 +288,7 @@ below. The following options control the authentication of Web UIs: - + @@ -280,6 +296,7 @@ The following options control the authentication of Web UIs: See the Spark UI configuration for how to configure filters. + @@ -289,6 +306,7 @@ The following options control the authentication of Web UIs: permissions to view or modify the application. Note this requires the user to be authenticated, so if no authentication filter is installed, this option does not do anything. + @@ -296,6 +314,7 @@ The following options control the authentication of Web UIs: + @@ -303,6 +322,7 @@ The following options control the authentication of Web UIs: + @@ -310,6 +330,7 @@ The following options control the authentication of Web UIs: + @@ -317,6 +338,7 @@ The following options control the authentication of Web UIs: + @@ -324,6 +346,7 @@ The following options control the authentication of Web UIs: + @@ -331,6 +354,7 @@ The following options control the authentication of Web UIs: + @@ -347,6 +371,7 @@ The following options control the authentication of Web UIs: Windows environment is currently not supported. However, a new platform/protocol can be supported by implementing the trait mentioned above. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.ui.filters None1.0.0
    spark.acls.enable1.1.0
    spark.admin.acls Comma-separated list of users that have view and modify access to the Spark application. 1.1.0
    spark.admin.acls.groups Comma-separated list of groups that have view and modify access to the Spark application. 2.0.0
    spark.modify.acls Comma-separated list of users that have modify access to the Spark application. 1.1.0
    spark.modify.acls.groups Comma-separated list of groups that have modify access to the Spark application. 2.0.0
    spark.ui.view.acls Comma-separated list of users that have view access to the Spark application. 1.0.0
    spark.ui.view.acls.groups Comma-separated list of groups that have view access to the Spark application. 2.0.0
    spark.user.groups.mapping2.0.0
    @@ -361,7 +386,7 @@ servlet filters. To enable authorization in the SHS, a few extra options are used: - + @@ -375,6 +400,7 @@ To enable authorization in the SHS, a few extra options are used: If disabled, no access control checks are made for any application UIs available through the history server. + @@ -383,6 +409,7 @@ To enable authorization in the SHS, a few extra options are used: Comma separated list of users that have view access to all the Spark applications in history server. + @@ -391,6 +418,7 @@ To enable authorization in the SHS, a few extra options are used: Comma separated list of groups that have view access to all the Spark applications in history server. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.history.ui.acls.enable false1.0.1
    spark.history.ui.admin.acls2.1.1
    spark.history.ui.admin.acls.groups2.1.1
    @@ -606,7 +634,7 @@ Apache Spark can be configured to include HTTP headers to aid in preventing Cros Security. - + @@ -621,6 +649,7 @@ Security. of the page if an attack is detected.) + @@ -628,7 +657,8 @@ Security. - + + @@ -642,6 +672,7 @@ Security.
  • max-age=<expire-time>; preload
  • +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.ui.xXssProtection 1; mode=block 2.3.0
    spark.ui.xContentTypeOptions.enabled When enabled, X-Content-Type-Options HTTP response header will be set to "nosniff".
    2.3.0
    spark.ui.strictTransportSecurity None 2.3.0
    @@ -782,16 +813,17 @@ deployment-specific page for more information. The following options provides finer-grained control for this feature: - + + @@ -804,6 +836,7 @@ The following options provides finer-grained control for this feature: 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. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.security.credentials.${service}.enabled true - Controls whether to obtain credentials for services when security is enabled. - By default, credentials for all supported services are retrieved when those services are - configured, but it's possible to disable that behavior if it somehow conflicts with the - application being run. + Controls whether to obtain credentials for services when security is enabled. + By default, credentials for all supported services are retrieved when those services are + configured, but it's possible to disable that behavior if it somehow conflicts with the + application being run. 2.3.0
    spark.kerberos.access.hadoopFileSystems3.0.0
    diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 4d4b85e31c8ff..2c2ed53b478c3 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -192,6 +192,7 @@ SPARK_MASTER_OPTS supports the following system properties: The maximum number of completed applications to display. Older applications will be dropped from the UI to maintain this limit.
    + 0.8.0 spark.deploy.retainedDrivers @@ -199,6 +200,7 @@ SPARK_MASTER_OPTS supports the following system properties: The maximum number of completed drivers to display. Older drivers will be dropped from the UI to maintain this limit.
    + 1.1.0 spark.deploy.spreadOut @@ -208,6 +210,7 @@ SPARK_MASTER_OPTS supports the following system properties: to consolidate them onto as few nodes as possible. Spreading out is usually better for data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
    + 0.6.1 spark.deploy.defaultCores @@ -219,6 +222,7 @@ SPARK_MASTER_OPTS supports the following system properties: Set this lower on a shared cluster to prevent users from grabbing the whole cluster by default.
    + 0.9.0 spark.deploy.maxExecutorRetries @@ -234,6 +238,7 @@ SPARK_MASTER_OPTS supports the following system properties: -1.
    + 1.6.3 spark.worker.timeout @@ -250,6 +255,7 @@ SPARK_MASTER_OPTS supports the following system properties: Amount of a particular resource to use on the worker. + 3.0.0 spark.worker.resource.{resourceName}.discoveryScript @@ -258,6 +264,7 @@ SPARK_MASTER_OPTS supports the following system properties: Path to resource discovery script, which is used to find a particular resource while worker starting up. And the output of the script should be formatted like the ResourceInformation class. + 3.0.0 spark.worker.resourcesFile @@ -317,6 +324,7 @@ SPARK_WORKER_OPTS supports the following system properties: enabled). You should also enable spark.worker.cleanup.enabled, to ensure that the state eventually gets cleaned up. This config may be removed in the future. + 3.0.0 spark.storage.cleanupFilesAfterExecutorExit @@ -329,6 +337,7 @@ SPARK_WORKER_OPTS supports the following system properties: all files/subdirectories of a stopped and timeout application. This only affects Standalone mode, support of other cluster manangers can be added in the future. + 2.4.0 spark.worker.ui.compressedLogFileLengthCacheSize @@ -490,14 +499,16 @@ ZooKeeper is the best way to go for production-level high availability, but if y In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration: - + + +
    System propertyMeaning
    System propertyMeaningSince Version
    spark.deploy.recoveryMode Set to FILESYSTEM to enable single-node recovery mode (default: NONE).0.8.1
    spark.deploy.recoveryDirectory The directory in which Spark will store recovery state, accessible from the Master's perspective.0.8.1
    diff --git a/docs/sparkr-migration-guide.md b/docs/sparkr-migration-guide.md index 6fbc4c03aefc1..32836cdac53d5 100644 --- a/docs/sparkr-migration-guide.md +++ b/docs/sparkr-migration-guide.md @@ -28,8 +28,7 @@ Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide. ## Upgrading from SparkR 2.4 to 3.0 - - The deprecated methods `sparkR.init`, `sparkRSQL.init`, `sparkRHive.init` have been removed. Use `sparkR.session` instead. - - The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, `registerTempTable`, `createExternalTable`, and `dropTempTable` have been removed. Use `read.parquet`, `write.parquet`, `read.json`, `createOrReplaceTempView`, `createTable`, `dropTempView`, `union` instead. + - The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, `jsonRDD` have been removed. Use `read.parquet`, `write.parquet`, `read.json` instead. ## Upgrading from SparkR 2.3 to 2.4 diff --git a/docs/sparkr.md b/docs/sparkr.md index 24fa3b4feac19..d937bc90b6506 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -9,9 +9,9 @@ license: | 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. @@ -540,12 +540,15 @@ SparkR supports the following machine learning algorithms currently: * [`spark.mlp`](api/R/spark.mlp.html): [`Multilayer Perceptron (MLP)`](ml-classification-regression.html#multilayer-perceptron-classifier) * [`spark.naiveBayes`](api/R/spark.naiveBayes.html): [`Naive Bayes`](ml-classification-regression.html#naive-bayes) * [`spark.svmLinear`](api/R/spark.svmLinear.html): [`Linear Support Vector Machine`](ml-classification-regression.html#linear-support-vector-machine) +* [`spark.fmClassifier`](api/R/fmClassifier.html): [`Factorization Machines classifier`](ml-classification-regression.html#factorization-machines-classifier) #### Regression * [`spark.survreg`](api/R/spark.survreg.html): [`Accelerated Failure Time (AFT) Survival Model`](ml-classification-regression.html#survival-regression) * [`spark.glm`](api/R/spark.glm.html) or [`glm`](api/R/glm.html): [`Generalized Linear Model (GLM)`](ml-classification-regression.html#generalized-linear-regression) * [`spark.isoreg`](api/R/spark.isoreg.html): [`Isotonic Regression`](ml-classification-regression.html#isotonic-regression) +* [`spark.lm`](api/R/spark.lm.html): [`Linear Regression`](ml-classification-regression.html#linear-regression) +* [`spark.fmRegressor`](api/R/spark.fmRegressor.html): [`Factorization Machines regressor`](ml-classification-regression.html#factorization-machines-regressor) #### Tree @@ -756,4 +759,3 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma # Migration Guide The migration guide is now archived [on this page](sparkr-migration-guide.html). - diff --git a/docs/sql-data-sources-avro.md b/docs/sql-data-sources-avro.md index 8e6a4079cd5de..d926ae7703268 100644 --- a/docs/sql-data-sources-avro.md +++ b/docs/sql-data-sources-avro.md @@ -258,21 +258,34 @@ Data source options of Avro can be set via: ## Configuration Configuration of Avro can be done using the `setConf` method on SparkSession or by running `SET key=value` commands using SQL. - + - + + - + + - + +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.sql.legacy.replaceDatabricksSparkAvro.enabled trueIf it is set to true, the data source provider com.databricks.spark.avro is mapped to the built-in but external Avro data source module for backward compatibility. + If it is set to true, the data source provider com.databricks.spark.avro is mapped + to the built-in but external Avro data source module for backward compatibility. + 2.4.0
    spark.sql.avro.compression.codec snappyCompression codec used in writing of AVRO files. Supported codecs: uncompressed, deflate, snappy, bzip2 and xz. Default codec is snappy. + Compression codec used in writing of AVRO files. Supported codecs: uncompressed, deflate, + snappy, bzip2 and xz. Default codec is snappy. + 2.4.0
    spark.sql.avro.deflate.level -1Compression level for the deflate codec used in writing of AVRO files. Valid value must be in the range of from 1 to 9 inclusive or -1. The default value is -1 which corresponds to 6 level in the current implementation. + Compression level for the deflate codec used in writing of AVRO files. Valid value must be in + the range of from 1 to 9 inclusive or -1. The default value is -1 which corresponds to 6 level + in the current implementation. + 2.4.0
    diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md index 0054d466db987..22514cd082ecf 100644 --- a/docs/sql-data-sources-hive-tables.md +++ b/docs/sql-data-sources-hive-tables.md @@ -124,7 +124,7 @@ will compile against built-in Hive and use those classes for internal execution The following options can be used to configure the version of Hive that is used to retrieve metadata: - + @@ -132,6 +132,7 @@ The following options can be used to configure the version of Hive that is used Version of the Hive metastore. Available options are 0.12.0 through 2.3.6 and 3.0.0 through 3.1.2. + @@ -153,6 +154,7 @@ The following options can be used to configure the version of Hive that is used they are packaged with your application. + @@ -166,6 +168,7 @@ The following options can be used to configure the version of Hive that is used custom appenders that are used by log4j.

    + @@ -177,5 +180,6 @@ The following options can be used to configure the version of Hive that is used prefix that typically would be shared (i.e. org.apache.spark.*).

    +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.sql.hive.metastore.version 2.3.61.4.0
    spark.sql.hive.metastore.jars 1.4.0
    spark.sql.hive.metastore.sharedPrefixes 1.4.0
    spark.sql.hive.metastore.barrierPrefixes 1.4.0
    diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index 10d52bfbf7c05..b95be0974585e 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -231,7 +231,7 @@ the following case-insensitive options: {% include_example jdbc_dataset r/RSparkSQLExample.R %} -
    +
    {% highlight sql %} diff --git a/docs/sql-data-sources-json.md b/docs/sql-data-sources-json.md index 588f6cdaa0ad5..0f1ca432b704a 100644 --- a/docs/sql-data-sources-json.md +++ b/docs/sql-data-sources-json.md @@ -77,7 +77,7 @@ For a regular multi-line JSON file, set a named parameter `multiLine` to `TRUE`.
    -
    +
    {% highlight sql %} diff --git a/docs/sql-data-sources-load-save-functions.md b/docs/sql-data-sources-load-save-functions.md index a7efb9347ac64..0866f37fbdc71 100644 --- a/docs/sql-data-sources-load-save-functions.md +++ b/docs/sql-data-sources-load-save-functions.md @@ -127,7 +127,7 @@ visit the official Apache ORC/Parquet websites. {% include_example manual_save_options_orc r/RSparkSQLExample.R %}
    -
    +
    {% highlight sql %} CREATE TABLE users_with_options ( @@ -257,7 +257,7 @@ Bucketing and sorting are applicable only to persistent tables: {% include_example write_sorting_and_bucketing python/sql/datasource.py %}
    -
    +
    {% highlight sql %} @@ -291,7 +291,7 @@ while partitioning can be used with both `save` and `saveAsTable` when using the {% include_example write_partitioning python/sql/datasource.py %}
    -
    +
    {% highlight sql %} @@ -323,7 +323,7 @@ It is possible to use both partitioning and bucketing for a single table: {% include_example write_partition_and_bucket python/sql/datasource.py %}
    -
    +
    {% highlight sql %} diff --git a/docs/sql-data-sources-orc.md b/docs/sql-data-sources-orc.md index bddffe02602e8..4c4b3b1eee8c2 100644 --- a/docs/sql-data-sources-orc.md +++ b/docs/sql-data-sources-orc.md @@ -27,15 +27,25 @@ serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileF the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`. - + - + + - + +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.sql.orc.impl nativeThe name of ORC implementation. It can be one of native and hive. native means the native ORC support. hive means the ORC library in Hive. + The name of ORC implementation. It can be one of native and hive. + native means the native ORC support. hive means the ORC library + in Hive. + 2.3.0
    spark.sql.orc.enableVectorizedReader trueEnables vectorized orc decoding in native implementation. If false, a new non-vectorized ORC reader is used in native implementation. For hive implementation, this is ignored. + Enables vectorized orc decoding in native implementation. If false, + a new non-vectorized ORC reader is used in native implementation. + For hive implementation, this is ignored. + 2.3.0
    diff --git a/docs/sql-data-sources-parquet.md b/docs/sql-data-sources-parquet.md index 53a1111cd8286..7875b10b0f697 100644 --- a/docs/sql-data-sources-parquet.md +++ b/docs/sql-data-sources-parquet.md @@ -52,7 +52,7 @@ Using the data from the above example:
    -
    +
    {% highlight sql %} @@ -242,7 +242,7 @@ refreshTable("my_table")
    -
    +
    {% highlight sql %} REFRESH TABLE my_table; @@ -258,7 +258,7 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession `SET key=value` commands using SQL. - + @@ -267,6 +267,7 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession not differentiate between binary data and strings when writing out the Parquet schema. This flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. + @@ -275,6 +276,7 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems. + @@ -287,11 +289,13 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession Note that zstd requires ZStandardCodec to be installed before Hadoop 2.9.0, brotli requires BrotliCodec to be installed. + + @@ -300,6 +304,7 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in support. + @@ -310,6 +315,7 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession schema is picked from the summary file or a random data file if no summary file is available.

    + @@ -321,5 +327,6 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession example, decimals will be written in int-based format. If Parquet output is intended for use with systems that do not support this newer format, set to true. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.sql.parquet.binaryAsString false1.1.1
    spark.sql.parquet.int96AsTimestamp1.3.0
    spark.sql.parquet.compression.codec1.1.1
    spark.sql.parquet.filterPushdown true Enables Parquet filter push-down optimization when set to true.1.2.0
    spark.sql.hive.convertMetastoreParquet1.1.1
    spark.sql.parquet.mergeSchema 1.5.0
    spark.sql.parquet.writeLegacyFormat1.6.0
    diff --git a/docs/sql-getting-started.md b/docs/sql-getting-started.md index 9df0f768268d0..dab34afaaabb2 100644 --- a/docs/sql-getting-started.md +++ b/docs/sql-getting-started.md @@ -205,7 +205,7 @@ refer it, e.g. `SELECT * FROM global_temp.view1`. {% include_example global_temp_view python/sql/basic.py %}
    -
    +
    {% highlight sql %} @@ -356,31 +356,8 @@ aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, et While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in [Scala](api/scala/org/apache/spark/sql/expressions/scalalang/typed$.html) 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. +Moreover, users are not limited to the predefined aggregate functions and can create their own. For more details +about user defined aggregate functions, please refer to the documentation of +[User Defined Aggregate Functions](sql-ref-functions-udf-aggregate.html). -### Type-Safe User-Defined Aggregate Functions -User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/org/apache/spark/sql/expressions/Aggregator.html) 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%} -
    -
    - -### Untyped User-Defined Aggregate Functions -Typed aggregations, as described above, may also be registered as untyped aggregating UDFs for use with DataFrames. -For example, a user-defined average for untyped DataFrames 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%} -
    -
    diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index d6b663df9b882..db24299ad25ac 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -23,335 +23,198 @@ license: | {:toc} ## Upgrading from Spark SQL 3.0 to 3.1 - - Since Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.integerGroupingId` to `true`. + + - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.integerGroupingId` to `true`. + + - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.0, you can set `spark.sql.ui.explainMode` to `extended`. ## Upgrading from Spark SQL 2.4 to 3.0 ### Dataset/DataFrame APIs - - Since Spark 3.0, the Dataset and DataFrame API `unionAll` is not deprecated any more. It is an alias for `union`. + - In Spark 3.0, the Dataset and DataFrame API `unionAll` is no longer deprecated. It is an alias for `union`. - - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a grouped dataset with key attribute wrongly named as "value", if the key is non-struct type, e.g. int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries weird. For example, the schema of `ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the grouping attribute to "key". The old behaviour is preserved under a newly added configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`. + - In Spark 2.4 and below, `Dataset.groupByKey` results to a grouped dataset with key attribute is wrongly named as "value", if the key is non-struct type, for example, int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries unexpected. For example, the schema of `ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the grouping attribute to "key". The old behavior is preserved under a newly added configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`. ### DDL Statements - - Since Spark 3.0, `CREATE TABLE` without a specific provider will use the value of `spark.sql.sources.default` as its provider. In Spark version 2.4 and earlier, it was hive. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.createHiveTableByDefault.enabled` to `true`. + - In Spark 3.0, `CREATE TABLE` without a specific provider uses the value of `spark.sql.sources.default` as its provider. In Spark version 2.4 and below, it was Hive. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.createHiveTableByDefault.enabled` to `true`. - - Since Spark 3.0, when inserting a value into a table column with a different data type, the type coercion is performed as per ANSI SQL standard. Certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean` are disallowed. A runtime exception will be thrown if the value is out-of-range for the data type of the column. In Spark version 2.4 and earlier, type conversions during table insertion are allowed as long as they are valid `Cast`. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of byte type, the result is 1. The behavior is controlled by the option `spark.sql.storeAssignmentPolicy`, with a default value as "ANSI". Setting the option as "Legacy" restores the previous behavior. + - In Spark 3.0, when inserting a value into a table column with a different data type, the type coercion is performed as per ANSI SQL standard. Certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean` are disallowed. A runtime exception is thrown if the value is out-of-range for the data type of the column. In Spark version 2.4 and below, type conversions during table insertion are allowed as long as they are valid `Cast`. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of byte type, the result is 1. The behavior is controlled by the option `spark.sql.storeAssignmentPolicy`, with a default value as "ANSI". Setting the option as "Legacy" restores the previous behavior. - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set. - - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. + - Spark 2.4 and below: the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. In 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. - - Refreshing a cached table would trigger a table uncache operation and then a table cache (lazily) operation. In Spark version 2.4 and earlier, the cache name and storage level are not preserved before the uncache operation. Therefore, the cache name and storage level could be changed unexpectedly. Since Spark 3.0, cache name and storage level will be first preserved for cache recreation. It helps to maintain a consistent cache behavior upon table refreshing. + - Refreshing a cached table would trigger a table uncache operation and then a table cache (lazily) operation. In Spark version 2.4 and below, the cache name and storage level are not preserved before the uncache operation. Therefore, the cache name and storage level could be changed unexpectedly. In Spark 3.0, cache name and storage level are first preserved for cache recreation. It helps to maintain a consistent cache behavior upon table refreshing. - - Since Spark 3.0, the properties listing below become reserved, commands will fail if we specify reserved properties in places like `CREATE DATABASE ... WITH DBPROPERTIES` and `ALTER TABLE ... SET TBLPROPERTIES`. We need their specific clauses to specify them, e.g. `CREATE DATABASE test COMMENT 'any comment' LOCATION 'some path'`. We can set `spark.sql.legacy.notReserveProperties` to `true` to ignore the `ParseException`, in this case, these properties will be silently removed, e.g `SET DBPROTERTIES('location'='/tmp')` will affect nothing. In Spark version 2.4 and earlier, these properties are neither reserved nor have side effects, e.g. `SET DBPROTERTIES('location'='/tmp')` will not change the location of the database but only create a headless property just like `'a'='b'`. - - - - - - - - - - - - - - - - - - - - - - - - - -
    - Property(case sensitive) - - Database Reserved - - Table Reserved - - Remarks -
    - provider - - no - - yes - - For tables, please use the USING clause to specify it. Once set, it can't be changed. -
    - location - - yes - - yes - - For databases and tables, please use the LOCATION clause to specify it. -
    - owner - - yes - - yes - - For databases and tables, it is determined by the user who runs spark and create the table. -
    + - In Spark 3.0, the properties listing below become reserved; commands fail if you specify reserved properties in places like `CREATE DATABASE ... WITH DBPROPERTIES` and `ALTER TABLE ... SET TBLPROPERTIES`. You need their specific clauses to specify them, for example, `CREATE DATABASE test COMMENT 'any comment' LOCATION 'some path'`. You can set `spark.sql.legacy.notReserveProperties` to `true` to ignore the `ParseException`, in this case, these properties will be silently removed, for example: `SET DBPROTERTIES('location'='/tmp')` will have no effect. In Spark version 2.4 and below, these properties are neither reserved nor have side effects, for example, `SET DBPROTERTIES('location'='/tmp')` do not change the location of the database but only create a headless property just like `'a'='b'`. + + | Property (case sensitive) | Database Reserved | Table Reserved | Remarks | + | ------------------------- | ----------------- | -------------- | ------- | + | provider | no | yes | For tables, use the `USING` clause to specify it. Once set, it can't be changed. | + | location | yes | yes | For databases and tables, use the `LOCATION` clause to specify it. | + | owner | yes | yes | For databases and tables, it is determined by the user who runs spark and create the table. | + + + - In Spark 3.0, you can use `ADD FILE` to add file directories as well. Earlier you could add only single files using this command. To restore the behavior of earlier versions, set `spark.sql.legacy.addSingleFileInAddFile` to `true`. - - Since Spark 3.0, `ADD FILE` can be used to add file directories as well. Earlier only single files can be added using this command. To restore the behaviour of earlier versions, set `spark.sql.legacy.addSingleFileInAddFile` to `true`. + - In Spark 3.0, `SHOW TBLPROPERTIES` throws `AnalysisException` if the table does not exist. In Spark version 2.4 and below, this scenario caused `NoSuchTableException`. Also, `SHOW TBLPROPERTIES` on a temporary view causes `AnalysisException`. In Spark version 2.4 and below, it returned an empty result. - - Since Spark 3.0, `SHOW TBLPROPERTIES` will cause `AnalysisException` if the table does not exist. In Spark version 2.4 and earlier, this scenario caused `NoSuchTableException`. Also, `SHOW TBLPROPERTIES` on a temporary view will cause `AnalysisException`. In Spark version 2.4 and earlier, it returned an empty result. + - In Spark 3.0, `SHOW CREATE TABLE` always returns Spark DDL, even when the given table is a Hive SerDe table. For generating Hive DDL, use `SHOW CREATE TABLE AS SERDE` command instead. - - Since Spark 3.0, `SHOW CREATE TABLE` will always return Spark DDL, even when the given table is a Hive serde table. For generating Hive DDL, please use `SHOW CREATE TABLE AS SERDE` command instead. + - In Spark 3.0, column of CHAR type is not allowed in non-Hive-Serde tables, and CREATE/ALTER TABLE commands will fail if CHAR type is detected. Please use STRING type instead. In Spark version 2.4 and below, CHAR type is treated as STRING type and the length parameter is simply ignored. ### UDFs and Built-in Functions - - Since Spark 3.0, the `date_add` and `date_sub` functions only accepts int, smallint, tinyint as the 2nd argument, fractional and string types are not valid anymore, e.g. `date_add(cast('1964-05-23' as date), '12.34')` will cause `AnalysisException`. In Spark version 2.4 and earlier, if the 2nd argument is fractional or string value, it will be coerced to int value, and the result will be a date value of `1964-06-04`. + - In Spark 3.0, the `date_add` and `date_sub` functions accepts only int, smallint, tinyint as the 2nd argument; fractional and non-literal strings are not valid anymore, for example: `date_add(cast('1964-05-23' as date), '12.34')` causes `AnalysisException`. Note that, string literals are still allowed, but Spark will throw `AnalysisException` if the string content is not a valid integer. In Spark version 2.4 and below, if the 2nd argument is fractional or string value, it is coerced to int value, and the result is a date value of `1964-06-04`. - - Since Spark 3.0, the function `percentile_approx` and its alias `approx_percentile` only accept integral value with range in `[1, 2147483647]` as its 3rd argument `accuracy`, fractional and string types are disallowed, e.g. `percentile_approx(10.0, 0.2, 1.8D)` will cause `AnalysisException`. In Spark version 2.4 and earlier, if `accuracy` is fractional or string value, it will be coerced to an int value, `percentile_approx(10.0, 0.2, 1.8D)` is operated as `percentile_approx(10.0, 0.2, 1)` which results in `10.0`. + - In Spark 3.0, the function `percentile_approx` and its alias `approx_percentile` only accept integral value with range in `[1, 2147483647]` as its 3rd argument `accuracy`, fractional and string types are disallowed, for example, `percentile_approx(10.0, 0.2, 1.8D)` causes `AnalysisException`. In Spark version 2.4 and below, if `accuracy` is fractional or string value, it is coerced to an int value, `percentile_approx(10.0, 0.2, 1.8D)` is operated as `percentile_approx(10.0, 0.2, 1)` which results in `10.0`. - - Since Spark 3.0, an analysis exception will be thrown when hash expressions are applied on elements of MapType. To restore the behavior before Spark 3.0, set `spark.sql.legacy.allowHashOnMapType` to `true`. + - In Spark 3.0, an analysis exception is thrown when hash expressions are applied on elements of `MapType`. To restore the behavior before Spark 3.0, set `spark.sql.legacy.allowHashOnMapType` to `true`. - - Since Spark 3.0, when the `array`/`map` function is called without any parameters, it returns an empty collection with `NullType` as element type. In Spark version 2.4 and earlier, it returns an empty collection with `StringType` as element type. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.createEmptyCollectionUsingStringType` to `true`. + - In Spark 3.0, when the `array`/`map` function is called without any parameters, it returns an empty collection with `NullType` as element type. In Spark version 2.4 and below, it returns an empty collection with `StringType` as element type. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.createEmptyCollectionUsingStringType` to `true`. - - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. + - In Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. - - In Spark version 2.4 and earlier, users can create map values with map type key via built-in function such as `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can use `map_entries` function to convert map to array> as a workaround. In addition, users can still read map values with map type key from data source or Java/Scala collections, though it is discouraged. + - In Spark version 2.4 and below, you can create map values with map type key via built-in function such as `CreateMap`, `MapFromArrays`, etc. In Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can use `map_entries` function to convert map to array> as a workaround. In addition, users can still read map values with map type key from data source or Java/Scala collections, though it is discouraged. - - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, Spark will throw RuntimeException while duplicated keys are found. Users can set `spark.sql.mapKeyDedupPolicy` to LAST_WIN to deduplicate map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be undefined. + - In Spark version 2.4 and below, you can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, for example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. In Spark 3.0, Spark throws `RuntimeException` when duplicated keys are found. You can set `spark.sql.mapKeyDedupPolicy` to `LAST_WIN` to deduplicate map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (for example, Parquet), the behavior is undefined. - - Since Spark 3.0, using `org.apache.spark.sql.functions.udf(AnyRef, DataType)` is not allowed by default. Set `spark.sql.legacy.allowUntypedScalaUDF` to true to keep using it. But please note that, in Spark version 2.4 and earlier, if `org.apache.spark.sql.functions.udf(AnyRef, DataType)` gets a Scala closure with primitive-type argument, the returned UDF will return null if the input values is null. However, since Spark 3.0, the UDF will return the default value of the Java type if the input value is null. For example, `val f = udf((x: Int) => x, IntegerType)`, `f($"x")` will return null in Spark 2.4 and earlier if column `x` is null, and return 0 in Spark 3.0. This behavior change is introduced because Spark 3.0 is built with Scala 2.12 by default. + - In Spark 3.0, using `org.apache.spark.sql.functions.udf(AnyRef, DataType)` is not allowed by default. Set `spark.sql.legacy.allowUntypedScalaUDF` to true to keep using it. In Spark version 2.4 and below, if `org.apache.spark.sql.functions.udf(AnyRef, DataType)` gets a Scala closure with primitive-type argument, the returned UDF returns null if the input values is null. However, in Spark 3.0, the UDF returns the default value of the Java type if the input value is null. For example, `val f = udf((x: Int) => x, IntegerType)`, `f($"x")` returns null in Spark 2.4 and below if column `x` is null, and return 0 in Spark 3.0. This behavior change is introduced because Spark 3.0 is built with Scala 2.12 by default. - - Since Spark 3.0, a higher-order function `exists` follows the three-valued boolean logic, i.e., if the `predicate` returns any `null`s and no `true` is obtained, then `exists` will return `null` instead of `false`. For example, `exists(array(1, null, 3), x -> x % 2 == 0)` will be `null`. The previous behaviour can be restored by setting `spark.sql.legacy.followThreeValuedLogicInArrayExists` to `false`. + - In Spark 3.0, a higher-order function `exists` follows the three-valued boolean logic, that is, if the `predicate` returns any `null`s and no `true` is obtained, then `exists` returns `null` instead of `false`. For example, `exists(array(1, null, 3), x -> x % 2 == 0)` is `null`. The previous behaviorcan be restored by setting `spark.sql.legacy.followThreeValuedLogicInArrayExists` to `false`. - - Since Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`. + - In Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and below, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`. - - In Spark version 2.4 and earlier, the `current_timestamp` function returns a timestamp with millisecond resolution only. Since Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution. + - In Spark version 2.4 and below, the `current_timestamp` function returns a timestamp with millisecond resolution only. In Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution. - - Since Spark 3.0, 0-argument Java UDF is executed in the executor side identically with other UDFs. In Spark version 2.4 and earlier, 0-argument Java UDF alone was executed in the driver side, and the result was propagated to executors, which might be more performant in some cases but caused inconsistency with a correctness issue in some cases. + - In Spark 3.0, a 0-argument Java UDF is executed in the executor side identically with other UDFs. In Spark version 2.4 and below, the 0-argument Java UDF alone was executed in the driver side, and the result was propagated to executors, which might be more performant in some cases but caused inconsistency with a correctness issue in some cases. - The result of `java.lang.Math`'s `log`, `log1p`, `exp`, `expm1`, and `pow` may vary across platforms. In Spark 3.0, the result of the equivalent SQL functions (including related SQL functions like `LOG10`) return values consistent with `java.lang.StrictMath`. In virtually all cases this makes no difference in the return value, and the difference is very small, but may not exactly match `java.lang.Math` on x86 platforms in cases like, for example, `log(3.0)`, whose value varies between `Math.log()` and `StrictMath.log()`. - - Since Spark 3.0, `Cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in case insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behaviour change is illustrated in the table below: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - Operation - - Result prior to Spark 3.0 - - Result starting Spark 3.0 -
    - CAST('infinity' AS DOUBLE)
    - CAST('+infinity' AS DOUBLE)
    - CAST('inf' AS DOUBLE)
    - CAST('+inf' AS DOUBLE)
    -
    - NULL - - Double.PositiveInfinity -
    - CAST('-infinity' AS DOUBLE)
    - CAST('-inf' AS DOUBLE)
    -
    - NULL - - Double.NegativeInfinity -
    - CAST('infinity' AS FLOAT)
    - CAST('+infinity' AS FLOAT)
    - CAST('inf' AS FLOAT)
    - CAST('+inf' AS FLOAT)
    -
    - NULL - - Float.PositiveInfinity -
    - CAST('-infinity' AS FLOAT)
    - CAST('-inf' AS FLOAT)
    -
    - NULL - - Float.NegativeInfinity -
    - CAST('nan' AS DOUBLE) - - NULL - - Double.NaN -
    - CAST('nan' AS FLOAT) - - NULL - - Float.NaN -
    + - In Spark 3.0, the `Cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in a case-insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behavior change is illustrated in the table below: + + | Operation | Result before Spark 3.0 | Result in Spark 3.0 | + | --------- | ----------------------- | ------------------- | + | CAST('infinity' AS DOUBLE) | NULL | Double.PositiveInfinity | + | CAST('+infinity' AS DOUBLE) | NULL | Double.PositiveInfinity | + | CAST('inf' AS DOUBLE) | NULL | Double.PositiveInfinity | + | CAST('inf' AS DOUBLE) | NULL | Double.PositiveInfinity | + | CAST('-infinity' AS DOUBLE) | NULL | Double.NegativeInfinity | + | CAST('-inf' AS DOUBLE) | NULL | Double.NegativeInfinity | + | CAST('infinity' AS FLOAT) | NULL | Float.PositiveInfinity | + | CAST('+infinity' AS FLOAT) | NULL | Float.PositiveInfinity | + | CAST('inf' AS FLOAT) | NULL | Float.PositiveInfinity | + | CAST('+inf' AS FLOAT) | NULL | Float.PositiveInfinity | + | CAST('-infinity' AS FLOAT) | NULL | Float.NegativeInfinity | + | CAST('-inf' AS FLOAT) | NULL | Float.NegativeInfinity | + | CAST('nan' AS DOUBLE) | NULL | Double.Nan | + | CAST('nan' AS FLOAT) | NULL | Float.NaN | + + - In Spark 3.0, when casting interval values to string type, there is no "interval" prefix, for example, `1 days 2 hours`. In Spark version 2.4 and below, the string contains the "interval" prefix like `interval 1 days 2 hours`. + + - In Spark 3.0, when casting string value to integral types(tinyint, smallint, int and bigint), datetime types(date, timestamp and interval) and boolean type, the leading and trailing whitespaces (<= ASCII 32) will be trimmed before converted to these type values, for example, `cast(' 1\t' as int)` results `1`, `cast(' 1\t' as boolean)` results `true`, `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and below, when casting string to integrals and booleans, it does not trim the whitespaces from both ends; the foregoing results is `null`, while to datetimes, only the trailing spaces (= ASCII 32) are removed. - - Since Spark 3.0, when casting interval values to string type, there is no "interval" prefix, e.g. `1 days 2 hours`. In Spark version 2.4 and earlier, the string contains the "interval" prefix like `interval 1 days 2 hours`. +### Query Engine - - Since Spark 3.0, when casting string value to integral types(tinyint, smallint, int and bigint), datetime types(date, timestamp and interval) and boolean type, the leading and trailing whitespaces (<= ASCII 32) will be trimmed before converted to these type values, e.g. `cast(' 1\t' as int)` results `1`, `cast(' 1\t' as boolean)` results `true`, `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, while casting string to integrals and booleans, it will not trim the whitespaces from both ends, the foregoing results will be `null`, while to datetimes, only the trailing spaces (= ASCII 32) will be removed. + - In Spark version 2.4 and below, SQL queries such as `FROM ` or `FROM
    UNION ALL FROM
    ` are supported by accident. In hive-style `FROM
    SELECT `, the `SELECT` clause is not negligible. Neither Hive nor Presto support this syntax. These queries are treated as invalid in Spark 3.0. -### Query Engine + - In Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception. - - In Spark version 2.4 and earlier, SQL queries such as `FROM
    ` or `FROM
    UNION ALL FROM
    ` are supported by accident. In hive-style `FROM
    SELECT `, the `SELECT` clause is not negligible. Neither Hive nor Presto support this syntax. Therefore we will treat these queries as invalid since Spark 3.0. + - In Spark 3.0, numbers written in scientific notation(for example, `1E2`) would be parsed as Double. In Spark version 2.4 and below, they're parsed as Decimal. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.exponentLiteralAsDecimal.enabled` to `true`. - - Since Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception. + - In Spark 3.0, day-time interval strings are converted to intervals with respect to the `from` and `to` bounds. If an input string does not match to the pattern defined by specified bounds, the `ParseException` exception is thrown. For example, `interval '2 10:20' hour to minute` raises the exception because the expected format is `[+|-]h[h]:[m]m`. In Spark version 2.4, the `from` bound was not taken into account, and the `to` bound was used to truncate the resulted interval. For instance, the day-time interval string from the showed example is converted to `interval 10 hours 20 minutes`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.fromDayTimeString.enabled` to `true`. - - Since Spark 3.0, numbers written in scientific notation(e.g. `1E2`) would be parsed as Double. In Spark version 2.4 and earlier, they're parsed as Decimal. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.exponentLiteralAsDecimal.enabled` to `true`. + - In Spark 3.0, negative scale of decimal is not allowed by default, for example, data type of literal like `1E10BD` is `DecimalType(11, 0)`. In Spark version 2.4 and below, it was `DecimalType(2, -9)`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.allowNegativeScaleOfDecimal` to `true`. - - Since Spark 3.0, day-time interval strings are converted to intervals with respect to the `from` and `to` bounds. If an input string does not match to the pattern defined by specified bounds, the `ParseException` exception is thrown. For example, `interval '2 10:20' hour to minute` raises the exception because the expected format is `[+|-]h[h]:[m]m`. In Spark version 2.4, the `from` bound was not taken into account, and the `to` bound was used to truncate the resulted interval. For instance, the day-time interval string from the showed example is converted to `interval 10 hours 20 minutes`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.fromDayTimeString.enabled` to `true`. - - - Since Spark 3.0, negative scale of decimal is not allowed by default, e.g. data type of literal like `1E10BD` is `DecimalType(11, 0)`. In Spark version 2.4 and earlier, it was `DecimalType(2, -9)`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.allowNegativeScaleOfDecimal` to `true`. + - In Spark 3.0, the unary arithmetic operator plus(`+`) only accepts string, numeric and interval type values as inputs. Besides, `+` with a integral string representation is coerced to a double value, for example, `+'1'` returns `1.0`. In Spark version 2.4 and below, this operator is ignored. There is no type checking for it, thus, all type values with a `+` prefix are valid, for example, `+ array(1, 2)` is valid and results `[1, 2]`. Besides, there is no type coercion for it at all, for example, in Spark 2.4, the result of `+'1'` is string `1`. - - Since Spark 3.0, the unary arithmetic operator plus(`+`) only accepts string, numeric and interval type values as inputs. Besides, `+` with a integral string representation will be coerced to double value, e.g. `+'1'` results `1.0`. In Spark version 2.4 and earlier, this operator is ignored. There is no type checking for it, thus, all type values with a `+` prefix are valid, e.g. `+ array(1, 2)` is valid and results `[1, 2]`. Besides, there is no type coercion for it at all, e.g. in Spark 2.4, the result of `+'1'` is string `1`. + - In Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`. - - Since Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`. + - In Spark 3.0, `spark.sql.legacy.ctePrecedencePolicy` is introduced to control the behavior for name conflicting in the nested WITH clause. By default value `EXCEPTION`, Spark throws an AnalysisException, it forces users to choose the specific substitution order they wanted. If set to `CORRECTED` (which is recommended), inner CTE definitions take precedence over outer definitions. For example, set the config to `false`, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `2`, while setting it to `LEGACY`, the result is `1` which is the behavior in version 2.4 and below. - - Since Spark 3.0, `spark.sql.legacy.ctePrecedencePolicy` is introduced to control the behavior for name conflicting in the nested WITH clause. By default value `EXCEPTION`, Spark throws an AnalysisException, it forces users to choose the specific substitution order they wanted. If set to `CORRECTED` (which is recommended), inner CTE definitions take precedence over outer definitions. For example, set the config to `false`, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `2`, while setting it to `LEGACY`, the result is `1` which is the behavior in version 2.4 and earlier. + - In Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join. - - Since Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join. + - In Spark version 2.4 and below, float/double -0.0 is semantically equal to 0.0, but -0.0 and 0.0 are considered as different values when used in aggregate grouping keys, window partition keys, and join keys. In Spark 3.0, this bug is fixed. For example, `Seq(-0.0, 0.0).toDF("d").groupBy("d").count()` returns `[(0.0, 2)]` in Spark 3.0, and `[(0.0, 1), (-0.0, 1)]` in Spark 2.4 and below. - - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal to 0.0, but -0.0 and 0.0 are considered as different values when used in aggregate grouping keys, window partition keys and join keys. Since Spark 3.0, this bug is fixed. For example, `Seq(-0.0, 0.0).toDF("d").groupBy("d").count()` returns `[(0.0, 2)]` in Spark 3.0, and `[(0.0, 1), (-0.0, 1)]` in Spark 2.4 and earlier. + - In Spark version 2.4 and below, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. In Spark 3.0, such time zone ids are rejected, and Spark throws `java.time.DateTimeException`. - - In Spark version 2.4 and earlier, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. Since Spark 3.0, such time zone ids are rejected, and Spark throws `java.time.DateTimeException`. + - In Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and so on. Spark 3.0 uses Java 8 API classes from the `java.time` packages that are based on [ISO chronology](https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and below, those operations are performed using the hybrid calendar ([Julian + Gregorian](https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API: - - Since Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and etc. Spark 3.0 uses Java 8 API classes from the java.time packages that based on ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and earlier, those operations are performed by using the hybrid calendar (Julian + Gregorian, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API: + * Parsing/formatting of timestamp/date strings. This effects on CSV/JSON datasources and on the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions when patterns specified by users is used for parsing and formatting. In Spark 3.0, we define our own pattern strings in `sql-ref-datetime-pattern.md`, which is implemented via `java.time.format.DateTimeFormatter` under the hood. New implementation performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. In Spark version 2.4 and below, `java.text.SimpleDateFormat` is used for timestamp/date string conversions, and the supported patterns are described in [simpleDateFormat](https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html). The old behavior can be restored by setting `spark.sql.legacy.timeParserPolicy` to `LEGACY`. - - Parsing/formatting of timestamp/date strings. This effects on CSV/JSON datasources and on the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions when patterns specified by users is used for parsing and formatting. Since Spark 3.0, we define our own pattern strings in `sql-ref-datetime-pattern.md`, which is implemented via `java.time.format.DateTimeFormatter` under the hood. New implementation performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. In Spark version 2.4 and earlier, `java.text.SimpleDateFormat` is used for timestamp/date string conversions, and the supported patterns are described in https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html. The old behavior can be restored by setting `spark.sql.legacy.timeParserPolicy` to `LEGACY`. + * The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone. - - The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone. + * The JDBC options `lowerBound` and `upperBound` are converted to TimestampType/DateType values in the same way as casting strings to TimestampType/DateType values. The conversion is based on Proleptic Gregorian calendar, and time zone defined by the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and below, the conversion is based on the hybrid calendar (Julian + Gregorian) and on default system time zone. - - the JDBC options `lowerBound` and `upperBound` are converted to TimestampType/DateType values in the same way as casting strings to TimestampType/DateType values. The conversion is based on Proleptic Gregorian calendar, and time zone defined by the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion is based on the hybrid calendar (Julian + Gregorian) and on default system time zone. + * Formatting `TIMESTAMP` and `DATE` literals. - - Formatting of `TIMESTAMP` and `DATE` literals. + * Creating typed `TIMESTAMP` and `DATE` literals from strings. In Spark 3.0, string conversion to typed `TIMESTAMP`/`DATE` literals is performed via casting to `TIMESTAMP`/`DATE` values. For example, `TIMESTAMP '2019-12-23 12:59:30'` is semantically equal to `CAST('2019-12-23 12:59:30' AS TIMESTAMP)`. When the input string does not contain information about time zone, the time zone from the SQL config `spark.sql.session.timeZone` is used in that case. In Spark version 2.4 and below, the conversion is based on JVM system time zone. The different sources of the default time zone may change the behavior of typed `TIMESTAMP` and `DATE` literals. - - Creating of typed `TIMESTAMP` and `DATE` literals from strings. Since Spark 3.0, string conversion to typed `TIMESTAMP`/`DATE` literals is performed via casting to `TIMESTAMP`/`DATE` values. For example, `TIMESTAMP '2019-12-23 12:59:30'` is semantically equal to `CAST('2019-12-23 12:59:30' AS TIMESTAMP)`. When the input string does not contain information about time zone, the time zone from the SQL config `spark.sql.session.timeZone` is used in that case. In Spark version 2.4 and earlier, the conversion is based on JVM system time zone. The different sources of the default time zone may change the behavior of typed `TIMESTAMP` and `DATE` literals. + - In Spark 3.0, `TIMESTAMP` literals are converted to strings using the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and below, the conversion uses the default time zone of the Java virtual machine. - - Since Spark 3.0, `TIMESTAMP` literals are converted to strings using the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion uses the default time zone of the Java virtual machine. + - In Spark 3.0, Spark casts `String` to `Date/Timestamp` in binary comparisons with dates/timestamps. The previous behavior of casting `Date/Timestamp` to `String` can be restored by setting `spark.sql.legacy.typeCoercion.datetimeToString.enabled` to `true`. - - Since Spark 3.0, Spark will cast `String` to `Date/TimeStamp` in binary comparisons with dates/timestamps. The previous behaviour of casting `Date/Timestamp` to `String` can be restored by setting `spark.sql.legacy.typeCoercion.datetimeToString.enabled` to `true`. + - In Spark 3.0, special values are supported in conversion from strings to dates and timestamps. Those values are simply notational shorthands that are converted to ordinary date or timestamp values when read. The following string values are supported for dates: + * `epoch [zoneId]` - 1970-01-01 + * `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone` + * `yesterday [zoneId]` - the current date - 1 + * `tomorrow [zoneId]` - the current date + 1 + * `now` - the date of running the current query. It has the same notion as today - - Since Spark 3.0, special values are supported in conversion from strings to dates and timestamps. Those values are simply notational shorthands that will be converted to ordinary date or timestamp values when read. The following string values are supported for dates: - - `epoch [zoneId]` - 1970-01-01 - - `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone` - - `yesterday [zoneId]` - the current date - 1 - - `tomorrow [zoneId]` - the current date + 1 - - `now` - the date of running the current query. It has the same notion as today - For example `SELECT date 'tomorrow' - date 'yesterday';` should output `2`. Here are special timestamp values: - - `epoch [zoneId]` - 1970-01-01 00:00:00+00 (Unix system time zero) - - `today [zoneId]` - midnight today - - `yesterday [zoneId]` - midnight yesterday - - `tomorrow [zoneId]` - midnight tomorrow - - `now` - current query start time - For example `SELECT timestamp 'tomorrow';`. + For example `SELECT date 'tomorrow' - date 'yesterday';` should output `2`. Here are special timestamp values: + * `epoch [zoneId]` - 1970-01-01 00:00:00+00 (Unix system time zero) + * `today [zoneId]` - midnight today + * `yesterday [zoneId]` - midnight yesterday + * `tomorrow [zoneId]` - midnight tomorrow + * `now` - current query start time + + For example `SELECT timestamp 'tomorrow';`. + + - Since Spark 3.0, when using `EXTRACT` expression to extract the second field from date/timestamp values, the result will be a `DecimalType(8, 6)` value with 2 digits for second part, and 6 digits for the fractional part with microsecond precision. e.g. `extract(second from to_timestamp('2019-09-20 10:10:10.1'))` results `10.100000`. In Spark version 2.4 and earlier, it returns an `IntegerType` value and the result for the former example is `10`. ### Data Sources - - In Spark version 2.4 and earlier, when reading a Hive Serde table with Spark native data sources(parquet/orc), Spark will infer the actual file schema and update the table schema in metastore. Since Spark 3.0, Spark doesn't infer the schema anymore. This should not cause any problems to end users, but if it does, please set `spark.sql.hive.caseSensitiveInferenceMode` to `INFER_AND_SAVE`. + - In Spark version 2.4 and below, when reading a Hive SerDe table with Spark native data sources(parquet/orc), Spark infers the actual file schema and update the table schema in metastore. In Spark 3.0, Spark doesn't infer the schema anymore. This should not cause any problems to end users, but if it does, set `spark.sql.hive.caseSensitiveInferenceMode` to `INFER_AND_SAVE`. - - In Spark version 2.4 and earlier, partition column value is converted as null if it can't be casted to corresponding user provided schema. Since 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`. + - In Spark version 2.4 and below, partition column value is converted as null if it can't be casted to corresponding user provided schema. In 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`. - - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. + - In Spark 3.0, if files or subdirectories disappear during recursive directory listing (that is, they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. - - In Spark version 2.4 and earlier, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType`, `DoubleType`, `DateType` and `TimestampType`, it fails on empty strings and throws exceptions. Since Spark 3.0, we disallow empty strings and will throw exceptions for data types except for `StringType` and `BinaryType`. The previous behaviour of allowing empty string can be restored by setting `spark.sql.legacy.json.allowEmptyString.enabled` to `true`. + - In Spark version 2.4 and below, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType`, `DoubleType`, `DateType` and `TimestampType`, it fails on empty strings and throws exceptions. Spark 3.0 disallows empty strings and will throw an exception for data types except for `StringType` and `BinaryType`. The previous behavior of allowing an empty string can be restored by setting `spark.sql.legacy.json.allowEmptyString.enabled` to `true`. - - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. + - In Spark version 2.4 and below, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. In Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. - - Since Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inferring. + - In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inference. - - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. + - In Spark version 2.4 and below, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. In Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. - - Since Spark 3.0, parquet logical type `TIMESTAMP_MICROS` is used by default while saving `TIMESTAMP` columns. In Spark version 2.4 and earlier, `TIMESTAMP` columns are saved as `INT96` in parquet files. Note that, some SQL systems such as Hive 1.x and Impala 2.x can only read `INT96` timestamps, you can set `spark.sql.parquet.outputTimestampType` as `INT96` to restore the previous behavior and keep interoperability. + - In Spark 3.0, parquet logical type `TIMESTAMP_MICROS` is used by default while saving `TIMESTAMP` columns. In Spark version 2.4 and below, `TIMESTAMP` columns are saved as `INT96` in parquet files. Note that, some SQL systems such as Hive 1.x and Impala 2.x can only read `INT96` timestamps, you can set `spark.sql.parquet.outputTimestampType` as `INT96` to restore the previous behavior and keep interoperability. - - Since Spark 3.0, when Avro files are written with user provided schema, the fields will be matched by field names between catalyst schema and avro schema instead of positions. + - In Spark 3.0, when Avro files are written with user provided schema, the fields are matched by field names between catalyst schema and Avro schema instead of positions. - - Since Spark 3.0, when Avro files are written with user provided non-nullable schema, even the catalyst schema is nullable, Spark is still able to write the files. However, Spark will throw runtime NPE if any of the records contains null. + - In Spark 3.0, when Avro files are written with user provided non-nullable schema, even the catalyst schema is nullable, Spark is still able to write the files. However, Spark throws runtime NullPointerException if any of the records contains null. ### Others - - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`. + - In Spark version 2.4, when a Spark session is created via `cloneSession()`, the newly created Spark session inherits its configuration from its parent `SparkContext` even though the same configuration may exist with a different value in its parent Spark session. In Spark 3.0, the configurations of a parent `SparkSession` have a higher precedence over the parent `SparkContext`. You can restore the old behavior by setting `spark.sql.legacy.sessionInitWithConfigDefaults` to `true`. - - In Spark 3.0, the deprecated `HiveContext` class has been removed. Use `SparkSession.builder.enableHiveSupport()` instead. + - In Spark 3.0, if `hive.default.fileformat` is not found in `Spark SQL configuration` then it falls back to the `hive-site.xml` file present in the `Hadoop configuration` of `SparkContext`. - - In Spark version 2.4, when a spark session is created via `cloneSession()`, the newly created spark session inherits its configuration from its parent `SparkContext` even though the same configuration may exist with a different value in its parent spark session. Since Spark 3.0, the configurations of a parent `SparkSession` have a higher precedence over the parent `SparkContext`. The old behavior can be restored by setting `spark.sql.legacy.sessionInitWithConfigDefaults` to `true`. + - In Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example: - - Since Spark 3.0, if `hive.default.fileformat` is not found in `Spark SQL configuration` then it will fallback to hive-site.xml present in the `Hadoop configuration` of `SparkContext`. + | Query | Spark 2.4 | Spark 3.0 | + | ----- | --------- | --------- | + |`SELECT CAST(1 AS decimal(38, 18));` | 1 | 1.000000000000000000 | - - Since Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example: -
    - - - - - - - - - - -
    - Query - - Spark 2.4 or Prior - - Spark 3.0 -
    - SELECT CAST(1 AS decimal(38, 18)); - - 1 - - 1.000000000000000000 -
    + - In Spark 3.0, we upgraded the built-in Hive from 1.2 to 2.3 and it brings following impacts: - - Since Spark 3.0, we upgraded the built-in Hive from 1.2 to 2.3 and it brings following impacts: - - - You may need to set `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars` according to the version of the Hive metastore you want to connect to. - For example: set `spark.sql.hive.metastore.version` to `1.2.1` and `spark.sql.hive.metastore.jars` to `maven` if your Hive metastore version is 1.2.1. - - - You need to migrate your custom SerDes to Hive 2.3 or build your own Spark with `hive-1.2` profile. See HIVE-15167 for more details. + * You may need to set `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars` according to the version of the Hive metastore you want to connect to. For example: set `spark.sql.hive.metastore.version` to `1.2.1` and `spark.sql.hive.metastore.jars` to `maven` if your Hive metastore version is 1.2.1. - - The decimal string representation can be different between Hive 1.2 and Hive 2.3 when using `TRANSFORM` operator in SQL for script transformation, which depends on hive's behavior. In Hive 1.2, the string representation omits trailing zeroes. But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary. + * You need to migrate your custom SerDes to Hive 2.3 or build your own Spark with `hive-1.2` profile. See [HIVE-15167](https://issues.apache.org/jira/browse/HIVE-15167) for more details. + + * The decimal string representation can be different between Hive 1.2 and Hive 2.3 when using `TRANSFORM` operator in SQL for script transformation, which depends on hive's behavior. In Hive 1.2, the string representation omits trailing zeroes. But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary. ## Upgrading from Spark SQL 2.4.4 to 2.4.5 @@ -967,29 +830,29 @@ Spark SQL supports the vast majority of Hive features, such as: * Unions * Sub-queries * Sub-queries in the FROM Clause - + ```SELECT col FROM (SELECT a + b AS col FROM t1) t2``` * Sub-queries in WHERE Clause * Correlated or non-correlated IN and NOT IN statement in WHERE Clause - + ``` SELECT col FROM t1 WHERE col IN (SELECT a FROM t2 WHERE t1.a = t2.a) SELECT col FROM t1 WHERE col IN (SELECT a FROM t2) ``` * Correlated or non-correlated EXISTS and NOT EXISTS statement in WHERE Clause - + ``` SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t1.a = t2.a AND t2.a > 10) SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t2.a > 10) ``` * Non-correlated IN and NOT IN statement in JOIN Condition - + ```SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND t1.a IN (SELECT a FROM t3)``` - + * Non-correlated EXISTS and NOT EXISTS statement in JOIN Condition - - ```SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND EXISTS (SELECT * FROM t3 WHERE t3.a > 10)``` - + + ```SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND EXISTS (SELECT * FROM t3 WHERE t3.a > 10)``` + * Sampling * Explain * Partitioned tables including dynamic partition insertion diff --git a/docs/sql-performance-tuning.md b/docs/sql-performance-tuning.md index 5a86c0cc31e12..7cd85b6a9ab4c 100644 --- a/docs/sql-performance-tuning.md +++ b/docs/sql-performance-tuning.md @@ -35,7 +35,7 @@ Configuration of in-memory caching can be done using the `setConf` method on `Sp `SET key=value` commands using SQL. - + @@ -43,6 +43,7 @@ Configuration of in-memory caching can be done using the `setConf` method on `Sp When set to true Spark SQL will automatically select a compression codec for each column based on statistics of the data. + @@ -51,6 +52,7 @@ Configuration of in-memory caching can be done using the `setConf` method on `Sp Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization and compression, but risk OOMs when caching data. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.sql.inMemoryColumnarStorage.compressed true1.0.1
    spark.sql.inMemoryColumnarStorage.batchSize1.1.1
    @@ -61,7 +63,7 @@ The following options can also be used to tune the performance of query executio that these options will be deprecated in future release as more optimizations are performed automatically. - + @@ -69,6 +71,7 @@ that these options will be deprecated in future release as more optimizations ar The maximum number of bytes to pack into a single partition when reading files. This configuration is effective only when using file-based sources such as Parquet, JSON and ORC. + @@ -80,15 +83,17 @@ that these options will be deprecated in future release as more optimizations ar scheduled first). This configuration is effective only when using file-based sources such as Parquet, JSON and ORC. + + @@ -99,6 +104,7 @@ that these options will be deprecated in future release as more optimizations ar statistics are only supported for Hive Metastore tables where the command ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run. + @@ -106,6 +112,7 @@ that these options will be deprecated in future release as more optimizations ar +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.sql.files.maxPartitionBytes 134217728 (128 MB)2.0.0
    spark.sql.files.openCostInBytes2.0.0
    spark.sql.broadcastTimeout 300 -

    - Timeout in seconds for the broadcast wait time in broadcast joins -

    +

    + Timeout in seconds for the broadcast wait time in broadcast joins +

    1.3.0
    spark.sql.autoBroadcastJoinThreshold1.1.0
    spark.sql.shuffle.partitions Configures the number of partitions to use when shuffling data for joins or aggregations. 1.1.0
    @@ -162,7 +169,7 @@ head(join(src, hint(records, "broadcast"), src$key == records$key))
    -
    +
    {% highlight sql %} -- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint @@ -172,6 +179,8 @@ SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key
    +For more details please refer to the documentation of [Join Hints](sql-ref-syntax-qry-select-hints.html). + ## Coalesce Hints for SQL Queries Coalesce hints allows the Spark SQL users to control the number of output files just like the @@ -186,3 +195,77 @@ The "REPARTITION_BY_RANGE" hint must have column names and a partition number is SELECT /*+ REPARTITION(3, c) */ * FROM t SELECT /*+ REPARTITION_BY_RANGE(c) */ * FROM t SELECT /*+ REPARTITION_BY_RANGE(3, c) */ * FROM t + +## Adaptive Query Execution +Adaptive Query Execution (AQE) is an optimization technique in Spark SQL that makes use of the runtime statistics to choose the most efficient query execution plan. AQE is disabled by default. Spark SQL can use the umbrella configuration of `spark.sql.adaptive.enabled` to control whether turn it on/off. As of Spark 3.0, there are three major features in AQE, including coalescing post-shuffle partitions, converting sort-merge join to broadcast join, and skew join optimization. + +### Coalescing Post Shuffle Partitions +This feature coalesces the post shuffle partitions based on the map output statistics when both `spark.sql.adaptive.enabled` and `spark.sql.adaptive.coalescePartitions.enabled` configurations are true. This feature simplifies the tuning of shuffle partition number when running queries. You do not need to set a proper shuffle partition number to fit your dataset. Spark can pick the proper shuffle partition number at runtime once you set a large enough initial number of shuffle partitions via `spark.sql.adaptive.coalescePartitions.initialPartitionNum` configuration. + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaningSince Version
    spark.sql.adaptive.coalescePartitions.enabledtrue + When true and spark.sql.adaptive.enabled is true, Spark will coalesce contiguous shuffle partitions according to the target size (specified by spark.sql.adaptive.advisoryPartitionSizeInBytes), to avoid too many small tasks. + 3.0.0
    spark.sql.adaptive.coalescePartitions.minPartitionNumDefault Parallelism + The minimum number of shuffle partitions after coalescing. If not set, the default value is the default parallelism of the Spark cluster. This configuration only has an effect when spark.sql.adaptive.enabled and spark.sql.adaptive.coalescePartitions.enabled are both enabled. + 3.0.0
    spark.sql.adaptive.coalescePartitions.initialPartitionNum200 + The initial number of shuffle partitions before coalescing. By default it equals to spark.sql.shuffle.partitions. This configuration only has an effect when spark.sql.adaptive.enabled and spark.sql.adaptive.coalescePartitions.enabled are both enabled. + 3.0.0
    spark.sql.adaptive.advisoryPartitionSizeInBytes64 MB + The advisory size in bytes of the shuffle partition during adaptive optimization (when spark.sql.adaptive.enabled is true). It takes effect when Spark coalesces small shuffle partitions or splits skewed shuffle partition. + 3.0.0
    + +### Converting sort-merge join to broadcast join +AQE converts sort-merge join to broadcast hash join when the runtime statistics of any join side is smaller than the broadcast hash join threshold. This is not as efficient as planning a broadcast hash join in the first place, but it's better than keep doing the sort-merge join, as we can save the sorting of both the join sides, and read shuffle files locally to save network traffic(if `spark.sql.adaptive.localShuffleReader.enabled` is true) + +### Optimizing Skew Join +Data skew can severely downgrade the performance of join queries. This feature dynamically handles skew in sort-merge join by splitting (and replicating if needed) skewed tasks into roughly evenly sized tasks. It takes effect when both `spark.sql.adaptive.enabled` and `spark.sql.adaptive.skewJoin.enabled` configurations are enabled. + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaningSince Version
    spark.sql.adaptive.skewJoin.enabledtrue + When true and spark.sql.adaptive.enabled is true, Spark dynamically handles skew in sort-merge join by splitting (and replicating if needed) skewed partitions. + 3.0.0
    spark.sql.adaptive.skewJoin.skewedPartitionFactor10 + A partition is considered as skewed if its size is larger than this factor multiplying the median partition size and also larger than spark.sql.adaptive.skewedPartitionThresholdInBytes. + 3.0.0
    spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes256MB + A partition is considered as skewed if its size in bytes is larger than this threshold and also larger than spark.sql.adaptive.skewJoin.skewedPartitionFactor multiplying the median partition size. Ideally this config should be set larger than spark.sql.adaptive.advisoryPartitionSizeInBytes. + 3.0.0
    diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md index e8abb9fed1b2b..a11162963b380 100644 --- a/docs/sql-pyspark-pandas-with-arrow.md +++ b/docs/sql-pyspark-pandas-with-arrow.md @@ -98,7 +98,7 @@ In the following sections, it describes the combinations of the supported type h The type hint can be expressed as `pandas.Series`, ... -> `pandas.Series`. -By using `pandas_udf` with the function having such type hints, it creates a Pandas UDF where the given +By using `pandas_udf` with the function having such type hints above, it creates a Pandas UDF where the given function takes one or more `pandas.Series` and outputs one `pandas.Series`. The output of the function should always be of the same length as the input. Internally, PySpark will execute a Pandas UDF by splitting columns into batches and calling the function for each batch as a subset of the data, then concatenating @@ -118,13 +118,15 @@ For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/p The type hint can be expressed as `Iterator[pandas.Series]` -> `Iterator[pandas.Series]`. -By using `pandas_udf` with the function having such type hints, it creates a Pandas UDF where the given -function takes an iterator of `pandas.Series` and outputs an iterator of `pandas.Series`. The output of each -series from the function should always be of the same length as the input. In this case, the created -Pandas UDF requires one input column when the Pandas UDF is called. To use multiple input columns, -a different type hint is required. See Iterator of Multiple Series to Iterator of Series. +By using `pandas_udf` with the function having such type hints above, it creates a Pandas UDF where the given +function takes an iterator of `pandas.Series` and outputs an iterator of `pandas.Series`. The +length of the entire output from the function should be the same length of the entire input; therefore, it can +prefetch the data from the input iterator as long as the lengths are the same. +In this case, the created Pandas UDF requires one input column when the Pandas UDF is called. To use +multiple input columns, a different type hint is required. See Iterator of Multiple Series to Iterator +of Series. -It is useful when the UDF execution requires initializing some states although internally it works +It is also useful when the UDF execution requires initializing some states although internally it works identically as Series to Series case. The pseudocode below illustrates the example. {% highlight python %} @@ -153,10 +155,11 @@ For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/p The type hint can be expressed as `Iterator[Tuple[pandas.Series, ...]]` -> `Iterator[pandas.Series]`. -By using `pandas_udf` with the function having such type hints, it creates a Pandas UDF where the +By using `pandas_udf` with the function having such type hints above, it creates a Pandas UDF where the given function takes an iterator of a tuple of multiple `pandas.Series` and outputs an iterator of `pandas.Series`. In this case, the created pandas UDF requires multiple input columns as many as the series in the tuple -when the Pandas UDF is called. It works identically as Iterator of Series to Iterator of Series case except the parameter difference. +when the Pandas UDF is called. Otherwise, it has the same characteristics and restrictions as Iterator of Series +to Iterator of Series case. The following example shows how to create this Pandas UDF: @@ -172,7 +175,7 @@ For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/p The type hint can be expressed as `pandas.Series`, ... -> `Any`. -By using `pandas_udf` with the function having such type hints, it creates a Pandas UDF similar +By using `pandas_udf` with the function having such type hints above, it creates a Pandas UDF similar to PySpark's aggregate functions. The given function takes `pandas.Series` and returns a scalar value. The return type should be a primitive data type, and the returned scalar can be either a python primitive type, e.g., `int` or `float` or a numpy data type, e.g., `numpy.int64` or `numpy.float64`. @@ -198,12 +201,14 @@ For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/p ## Pandas Function APIs -Pandas function APIs can directly apply a Python native function against the whole DataFrame by -using Pandas instances. Internally it works similarly with Pandas UDFs by Spark using Arrow to transfer -data and Pandas to work with the data, which allows vectorized operations. A Pandas function API behaves -as a regular API under PySpark `DataFrame` in general. +Pandas Function APIs can directly apply a Python native function against the whole `DataFrame` by +using Pandas instances. Internally it works similarly with Pandas UDFs by using Arrow to transfer +data and Pandas to work with the data, which allows vectorized operations. However, A Pandas Function +API behaves as a regular API under PySpark `DataFrame` instead of `Column`, and Python type hints in Pandas +Functions APIs are optional and do not affect how it works internally at this moment although they +might be required in the future. -From Spark 3.0, Grouped map pandas UDF is now categorized as a separate Pandas Function API, +From Spark 3.0, grouped map pandas UDF is now categorized as a separate Pandas Function API, `DataFrame.groupby().applyInPandas()`. It is still possible to use it with `PandasUDFType` and `DataFrame.groupby().apply()` as it was; however, it is preferred to use `DataFrame.groupby().applyInPandas()` directly. Using `PandasUDFType` will be deprecated diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index bcee87c3711ea..8df9645f098f8 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -21,14 +21,14 @@ license: | Since Spark 3.0, Spark SQL introduces two experimental options to comply with the SQL standard: `spark.sql.ansi.enabled` and `spark.sql.storeAssignmentPolicy` (See a table below for details). -When `spark.sql.ansi.enabled` is set to `true`, Spark SQL follows the standard in basic behaviours (e.g., arithmetic operations, type conversion, and SQL parsing). +When `spark.sql.ansi.enabled` is set to `true`, Spark SQL follows the standard in basic behaviours (e.g., arithmetic operations, type conversion, SQL functions and SQL parsing). Moreover, Spark SQL has an independent option to control implicit casting behaviours when inserting rows in a table. The casting behaviours are defined as store assignment rules in the standard. When `spark.sql.storeAssignmentPolicy` is set to `ANSI`, Spark SQL complies with the ANSI store assignment rules. This is a separate configuration because its default value is `ANSI`, while the configuration `spark.sql.ansi.enabled` is disabled by default. - + @@ -37,6 +37,7 @@ When `spark.sql.storeAssignmentPolicy` is set to `ANSI`, Spark SQL complies with 1. Spark will throw a runtime exception if an overflow occurs in any operation on integral/decimal field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in the SQL parser. + @@ -52,6 +53,7 @@ When `spark.sql.storeAssignmentPolicy` is set to `ANSI`, Spark SQL complies with With strict policy, Spark doesn't allow any possible precision loss or data truncation in type coercion, e.g. converting double to int or decimal to double is not allowed. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.sql.ansi.enabled false3.0.0
    spark.sql.storeAssignmentPolicy3.0.0
    @@ -67,18 +69,15 @@ When `spark.sql.ansi.enabled` is set to `true` and an overflow occurs in numeric {% highlight sql %} -- `spark.sql.ansi.enabled=true` SELECT 2147483647 + 1; - java.lang.ArithmeticException: integer overflow -- `spark.sql.ansi.enabled=false` SELECT 2147483647 + 1; - +----------------+ |(2147483647 + 1)| +----------------+ | -2147483648| +----------------+ - {% endhighlight %} ### Type Conversion @@ -95,16 +94,13 @@ In future releases, the behaviour of type coercion might change along with the o -- `spark.sql.ansi.enabled=true` SELECT CAST('a' AS INT); - java.lang.NumberFormatException: invalid input syntax for type numeric: a SELECT CAST(2147483648L AS INT); - java.lang.ArithmeticException: Casting 2147483648 to int causes overflow -- `spark.sql.ansi.enabled=false` (This is a default behaviour) SELECT CAST('a' AS INT); - +--------------+ |CAST(a AS INT)| +--------------+ @@ -112,7 +108,6 @@ SELECT CAST('a' AS INT); +--------------+ SELECT CAST(2147483648L AS INT); - +-----------------------+ |CAST(2147483648 AS INT)| +-----------------------+ @@ -124,22 +119,24 @@ CREATE TABLE t (v INT); -- `spark.sql.storeAssignmentPolicy=ANSI` INSERT INTO t VALUES ('1'); - org.apache.spark.sql.AnalysisException: Cannot write incompatible data to table '`default`.`t`': - Cannot safely cast 'v': StringType to IntegerType; -- `spark.sql.storeAssignmentPolicy=LEGACY` (This is a legacy behaviour until Spark 2.x) INSERT INTO t VALUES ('1'); SELECT * FROM t; - +---+ | v| +---+ | 1| +---+ - {% endhighlight %} +### SQL Functions + +The behavior of some SQL functions can be different under ANSI mode (`spark.sql.ansi.enabled=true`). + - `size`: This function returns null for null input under ANSI mode. + ### SQL Keywords When `spark.sql.ansi.enabled` is true, Spark SQL will use the ANSI mode parser. @@ -405,6 +402,7 @@ Below is a list of all the keywords in Spark SQL. USINGreservedstrict-non-reservedreserved VALUESnon-reservednon-reservedreserved VIEWnon-reservednon-reservednon-reserved + VIEWSnon-reservednon-reservednon-reserved WHENreservednon-reservedreserved WHEREreservednon-reservedreserved WINDOWnon-reservednon-reservedreserved diff --git a/docs/sql-ref-datatypes.md b/docs/sql-ref-datatypes.md index 9700608fe8a34..150e194ae6cde 100644 --- a/docs/sql-ref-datatypes.md +++ b/docs/sql-ref-datatypes.md @@ -19,7 +19,6 @@ license: | limitations under the License. --- - Spark SQL and DataFrames support the following data types: * Numeric types @@ -632,7 +631,7 @@ from pyspark.sql.types import *
    -
    +
    The following table shows the type names as well as aliases used in Spark SQL parser for each data type. diff --git a/docs/sql-ref-datetime-pattern.md b/docs/sql-ref-datetime-pattern.md index 429d781248ef2..80585a4fd484c 100644 --- a/docs/sql-ref-datetime-pattern.md +++ b/docs/sql-ref-datetime-pattern.md @@ -21,11 +21,13 @@ license: | There are several common scenarios for datetime usage in Spark: -- CSV/JSON datasources use the pattern string for parsing and formatting time content. +- CSV/JSON datasources use the pattern string for parsing and formatting datetime content. -- Datetime functions related to convert string to/from `DateType` or `TimestampType`. For example, unix_timestamp, date_format, to_unix_timestamp, from_unixtime, to_date, to_timestamp, from_utc_timestamp, to_utc_timestamp, etc. +- Datetime functions related to convert `StringType` to/from `DateType` or `TimestampType`. + For example, `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp`, `from_utc_timestamp`, `to_utc_timestamp`, etc. + +Spark uses pattern letters in the following table for date and timestamp parsing and formatting: -Spark uses the below letters in date and timestamp parsing and formatting: @@ -52,7 +54,7 @@ Spark uses the below letters in date and timestamp parsing and formatting: - + @@ -63,6 +65,12 @@ Spark uses the below letters in date and timestamp parsing and formatting: + + + + + + @@ -88,7 +96,7 @@ Spark uses the below letters in date and timestamp parsing and formatting: - + @@ -147,6 +155,12 @@ Spark uses the below letters in date and timestamp parsing and formatting: + + + + + + @@ -189,6 +203,18 @@ Spark uses the below letters in date and timestamp parsing and formatting: + + + + + + + + + + + +
    Symbol 189
    M M/L month-of-year number/text 7; 07; Jul; July; J number 28
    Q/q quarter-of-year number/text 3; 03; Q3; 3rd quarter
    Y week-based-year Tue; Tuesday; T
    e u localized day-of-week number/text 2; 02; Tue; Tuesday; T fraction 978
    V time-zone ID zone-id America/Los_Angeles; Z; -08:30
    z time-zone name literal '
    [ optional section start
    ] optional section end
    The count of pattern letters determines the format. @@ -199,11 +225,16 @@ The count of pattern letters determines the format. - Number/Text: If the count of pattern letters is 3 or greater, use the Text rules above. Otherwise use the Number rules above. -- Fraction: Outputs the micro-of-second field as a fraction-of-second. The micro-of-second value has six digits, thus the count of pattern letters is from 1 to 6. If it is less than 6, then the micro-of-second value is truncated, with only the most significant digits being output. +- Fraction: Use one or more (up to 9) contiguous `'S'` characters, e,g `SSSSSS`, to parse and format fraction of second. + For parsing, the acceptable fraction length can be [1, the number of contiguous 'S']. + For formatting, the fraction length would be padded to the number of contiguous 'S' with zeros. + Spark supports datetime of micro-of-second precision, which has up to 6 significant digits, but can parse nano-of-second with exceeded part truncated. - Year: The count of letters determines the minimum field width below which padding is used. If the count of letters is two, then a reduced two digit form is used. For printing, this outputs the rightmost two digits. For parsing, this will parse using the base value of 2000, resulting in a year within the range 2000 to 2099 inclusive. If the count of letters is less than four (but not two), then the sign is only output for negative years. Otherwise, the sign is output if the pad width is exceeded when 'G' is not present. -- Zone names: This outputs the display name of the time-zone ID. If the count of letters is one, two or three, then the short name is output. If the count of letters is four, then the full name is output. Five or more letters will fail. +- Zone ID(V): This outputs the display the time-zone ID. Pattern letter count must be 2. + +- Zone names(z): This outputs the display textual name of the time-zone ID. If the count of letters is one, two or three, then the short name is output. If the count of letters is four, then the full name is output. Five or more letters will fail. - Offset X and x: This formats the offset based on the number of pattern letters. One letter outputs just the hour, such as '+01', unless the minute is non-zero in which case the minute is also output, such as '+0130'. Two letters outputs the hour and minute, without a colon, such as '+0130'. Three letters outputs the hour and minute, with a colon, such as '+01:30'. Four letters outputs the hour and minute and optional second, without a colon, such as '+013015'. Five letters outputs the hour and minute and optional second, with a colon, such as '+01:30:15'. Six or more letters will fail. Pattern letter 'X' (upper case) will output 'Z' when the offset to be output would be zero, whereas pattern letter 'x' (lower case) will output '+00', '+0000', or '+00:00'. @@ -211,6 +242,11 @@ The count of pattern letters determines the format. - Offset Z: This formats the offset based on the number of pattern letters. One, two or three letters outputs the hour and minute, without a colon, such as '+0130'. The output will be '+0000' when the offset is zero. Four letters outputs the full form of localized offset, equivalent to four letters of Offset-O. The output will be the corresponding localized offset text if the offset is zero. Five letters outputs the hour, minute, with optional second if non-zero, with colon. It outputs 'Z' if the offset is zero. Six or more letters will fail. +- Optional section start and end: Use `[]` to define an optional section and maybe nested. + During formatting, all valid data will be output even it is in the optional section. + During parsing, the whole section may be missing from the parsed string. + An optional section is started by `[` and ended using `]` (or at the end of the pattern). + More details for the text style: - Short Form: Short text, typically an abbreviation. For example, day-of-week Monday might output "Mon". diff --git a/docs/sql-ref-functions-builtin-aggregate.md b/docs/sql-ref-functions-builtin-aggregate.md deleted file mode 100644 index 3fcd78245ca04..0000000000000 --- a/docs/sql-ref-functions-builtin-aggregate.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -layout: global -title: Builtin Aggregate Functions -displayTitle: Builtin Aggregate Functions -license: | - 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. ---- - -**This page is under construction** diff --git a/docs/sql-ref-functions-builtin-scalar.md b/docs/sql-ref-functions-builtin-scalar.md deleted file mode 100644 index 1d818a25c4ac1..0000000000000 --- a/docs/sql-ref-functions-builtin-scalar.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -layout: global -title: Builtin Scalar Functions -displayTitle: Builtin Scalar Functions -license: | - 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. ---- - -**This page is under construction** diff --git a/docs/sql-ref-functions-builtin.md b/docs/sql-ref-functions-builtin.md deleted file mode 100644 index 6c57b0d6fdf6f..0000000000000 --- a/docs/sql-ref-functions-builtin.md +++ /dev/null @@ -1,25 +0,0 @@ ---- -layout: global -title: Reference -displayTitle: Reference -license: | - 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. ---- - -Spark SQL is Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. diff --git a/docs/sql-ref-functions-udf-aggregate.md b/docs/sql-ref-functions-udf-aggregate.md index 49c7b5824dfd6..3fde94d6bc4bf 100644 --- a/docs/sql-ref-functions-udf-aggregate.md +++ b/docs/sql-ref-functions-udf-aggregate.md @@ -1,7 +1,7 @@ --- layout: global -title: User defined Aggregate Functions (UDAF) -displayTitle: User defined Aggregate Functions (UDAF) +title: User Defined Aggregate Functions (UDAFs) +displayTitle: User Defined Aggregate Functions (UDAFs) license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -9,9 +9,9 @@ license: | 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. @@ -19,4 +19,120 @@ license: | limitations under the License. --- -**This page is under construction** +### Description + +User-Defined Aggregate Functions (UDAFs) are user-programmable routines that act on multiple rows at once and return a single aggregated value as a result. This documentation lists the classes that are required for creating and registering UDAFs. It also contains examples that demonstrate how to define and register UDAFs in Scala and invoke them in Spark SQL. + +### Aggregator[-IN, BUF, OUT] + +A base class for user-defined aggregations, which can be used in Dataset operations to take all of the elements of a group and reduce them to a single value. + + * IN - The input type for the aggregation. + * BUF - The type of the intermediate value of the reduction. + * OUT - The type of the final output result. + +
    +
    bufferEncoder: Encoder[BUF]
    +
    + Specifies the Encoder for the intermediate value type. +
    +
    +
    +
    finish(reduction: BUF): OUT
    +
    + Transform the output of the reduction. +
    +
    +
    +
    merge(b1: BUF, b2: BUF): BUF
    +
    + Merge two intermediate values. +
    +
    +
    +
    outputEncoder: Encoder[OUT]
    +
    + Specifies the Encoder for the final output value type. +
    +
    +
    +
    reduce(b: BUF, a: IN): BUF
    +
    + Aggregate input value a into current intermediate value. For performance, the function may modify b and return it instead of constructing new object for b. +
    +
    +
    +
    zero: BUF
    +
    + The initial value of the intermediate result for this aggregation. +
    +
    + +### Examples + +#### Type-Safe User-Defined Aggregate Functions + +User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/org/apache/spark/sql/expressions/Aggregator.html) 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%} +
    +
    + +#### Untyped User-Defined Aggregate Functions + +Typed aggregations, as described above, may also be registered as untyped aggregating UDFs for use with DataFrames. +For example, a user-defined average for untyped DataFrames 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%} +
    +
    +{% highlight sql %} +-- Compile and place UDAF MyAverage in a JAR file called `MyAverage.jar` in /tmp. +CREATE FUNCTION myAverage AS 'MyAverage' USING JAR '/tmp/MyAverage.jar'; + +SHOW USER FUNCTIONS; +-- +------------------+ +-- | function| +-- +------------------+ +-- | default.myAverage| +-- +------------------+ + +CREATE TEMPORARY VIEW employees +USING org.apache.spark.sql.json +OPTIONS ( + path "examples/src/main/resources/employees.json" +); + +SELECT * FROM employees; +-- +-------+------+ +-- | name|salary| +-- +-------+------+ +-- |Michael| 3000| +-- | Andy| 4500| +-- | Justin| 3500| +-- | Berta| 4000| +-- +-------+------+ + +SELECT myAverage(salary) as average_salary FROM employees; +-- +--------------+ +-- |average_salary| +-- +--------------+ +-- | 3750.0| +-- +--------------+ +{% endhighlight %} +
    +
    + +### Related Statements + + * [Scalar User Defined Functions (UDFs)](sql-ref-functions-udf-scalar.html) + * [Integration with Hive UDFs/UDAFs/UDTFs](sql-ref-functions-udf-hive.html) diff --git a/docs/sql-ref-functions-udf-hive.md b/docs/sql-ref-functions-udf-hive.md new file mode 100644 index 0000000000000..a87266dec2ea1 --- /dev/null +++ b/docs/sql-ref-functions-udf-hive.md @@ -0,0 +1,108 @@ +--- +layout: global +title: Integration with Hive UDFs/UDAFs/UDTFs +displayTitle: Integration with Hive UDFs/UDAFs/UDTFs +license: | + 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. +--- + +### Description + +Spark SQL supports integration of Hive UDFs, UDAFs and UDTFs. Similar to Spark UDFs and UDAFs, Hive UDFs work on a single row as input and generate a single row as output, while Hive UDAFs operate on multiple rows and return a single aggregated row as a result. In addition, Hive also supports UDTFs (User Defined Tabular Functions) that act on one row as input and return multiple rows as output. To use Hive UDFs/UDAFs/UTFs, the user should register them in Spark, and then use them in Spark SQL queries. + +### Examples + +Hive has two UDF interfaces: [UDF](https://github.com/apache/hive/blob/master/udf/src/java/org/apache/hadoop/hive/ql/exec/UDF.java) and [GenericUDF](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java). +An example below uses [GenericUDFAbs](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAbs.java) derived from `GenericUDF`. + +{% highlight sql %} +-- Register `GenericUDFAbs` and use it in Spark SQL. +-- Note that, if you use your own programmed one, you need to add a JAR containig it +-- into a classpath, +-- e.g., ADD JAR yourHiveUDF.jar; +CREATE TEMPORARY FUNCTION testUDF AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs'; + +SELECT * FROM t; + +-----+ + |value| + +-----+ + | -1.0| + | 2.0| + | -3.0| + +-----+ + +SELECT testUDF(value) FROM t; + +--------------+ + |testUDF(value)| + +--------------+ + | 1.0| + | 2.0| + | 3.0| + +--------------+ +{% endhighlight %} + + +An example below uses [GenericUDTFExplode](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFExplode.java) derived from [GenericUDTF](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java). + +{% highlight sql %} +-- Register `GenericUDTFExplode` and use it in Spark SQL +CREATE TEMPORARY FUNCTION hiveUDTF + AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDTFExplode'; + +SELECT * FROM t; + +------+ + | value| + +------+ + |[1, 2]| + |[3, 4]| + +------+ + +SELECT hiveUDTF(value) FROM t; + +---+ + |col| + +---+ + | 1| + | 2| + | 3| + | 4| + +---+ +{% endhighlight %} + +Hive has two UDAF interfaces: [UDAF](https://github.com/apache/hive/blob/master/udf/src/java/org/apache/hadoop/hive/ql/exec/UDAF.java) and [GenericUDAFResolver](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver.java). +An example below uses [GenericUDAFSum](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java) derived from `GenericUDAFResolver`. + +{% highlight sql %} +-- Register `GenericUDAFSum` and use it in Spark SQL +CREATE TEMPORARY FUNCTION hiveUDAF + AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum'; + +SELECT * FROM t; + +---+-----+ + |key|value| + +---+-----+ + | a| 1| + | a| 2| + | b| 3| + +---+-----+ + +SELECT key, hiveUDAF(value) FROM t GROUP BY key; + +---+---------------+ + |key|hiveUDAF(value)| + +---+---------------+ + | b| 3| + | a| 3| + +---+---------------+ +{% endhighlight %} \ No newline at end of file diff --git a/docs/sql-ref-functions-udf-scalar.md b/docs/sql-ref-functions-udf-scalar.md index cee135b572aea..2cb25f275cb59 100644 --- a/docs/sql-ref-functions-udf-scalar.md +++ b/docs/sql-ref-functions-udf-scalar.md @@ -1,7 +1,7 @@ --- layout: global -title: User defined Scalar Functions (UDF) -displayTitle: User defined Scalar Functions (UDF) +title: Scalar User Defined Functions (UDFs) +displayTitle: Scalar User Defined Functions (UDFs) license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -9,9 +9,9 @@ license: | 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. @@ -19,4 +19,43 @@ license: | limitations under the License. --- -**This page is under construction** +### Description + +User-Defined Functions (UDFs) are user-programmable routines that act on one row. This documentation lists the classes that are required for creating and registering UDFs. It also contains examples that demonstrate how to define and register UDFs and invoke them in Spark SQL. + +### UserDefinedFunction + +To define the properties of a user-defined function, the user can use some of the methods defined in this class. +
    +
    asNonNullable(): UserDefinedFunction
    +
    + Updates UserDefinedFunction to non-nullable. +
    +
    +
    +
    asNondeterministic(): UserDefinedFunction
    +
    + Updates UserDefinedFunction to nondeterministic. +
    +
    +
    +
    withName(name: String): UserDefinedFunction
    +
    + Updates UserDefinedFunction with a given name. +
    +
    + +### Examples + +
    +
    +{% include_example udf_scalar scala/org/apache/spark/examples/sql/UserDefinedScalar.scala%} +
    +
    + {% include_example udf_scalar java/org/apache/spark/examples/sql/JavaUserDefinedScalar.java%} +
    +
    + +### Related Statements + * [User Defined Aggregate Functions (UDAFs)](sql-ref-functions-udf-aggregate.html) + * [Integration with Hive UDFs/UDAFs/UDTFs](sql-ref-functions-udf-hive.html) diff --git a/docs/sql-ref-functions-udf.md b/docs/sql-ref-functions-udf.md deleted file mode 100644 index 6c57b0d6fdf6f..0000000000000 --- a/docs/sql-ref-functions-udf.md +++ /dev/null @@ -1,25 +0,0 @@ ---- -layout: global -title: Reference -displayTitle: Reference -license: | - 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. ---- - -Spark SQL is Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. diff --git a/docs/sql-ref-functions.md b/docs/sql-ref-functions.md index 6c57b0d6fdf6f..e8a0353579301 100644 --- a/docs/sql-ref-functions.md +++ b/docs/sql-ref-functions.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Functions +displayTitle: Functions license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -9,9 +9,9 @@ license: | 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. @@ -19,7 +19,13 @@ license: | limitations under the License. --- -Spark SQL is Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +Spark SQL provides two function features to meet a wide range of user needs: built-in functions and user-defined functions (UDFs). +Built-in functions are commonly used routines that Spark SQL predefines and a complete list of the functions can be found in the [Built-in Functions](api/sql/) API document. UDFs allow users to define their own functions when the system’s built-in functions are not enough to perform the desired task. + +### UDFs (User-Defined Functions) + +User-Defined Functions (UDFs) are a feature of Spark SQL that allows users to define their own functions when the system's built-in functions are not enough to perform the desired task. To use UDFs in Spark SQL, users must first define the function, then register the function with Spark, and finally call the registered function. The User-Defined Functions can act on a single row or act on multiple rows at once. Spark SQL also supports integration of existing Hive implementations of UDFs, UDAFs and UDTFs. + + * [Scalar User-Defined Functions (UDFs)](sql-ref-functions-udf-scalar.html) + * [User-Defined Aggregate Functions (UDAFs)](sql-ref-functions-udf-aggregate.html) + * [Integration with Hive UDFs/UDAFs/UDTFs](sql-ref-functions-udf-hive.html) diff --git a/docs/sql-ref-null-semantics.md b/docs/sql-ref-null-semantics.md index 37b4081d6b27b..dc48a36cadb3c 100644 --- a/docs/sql-ref-null-semantics.md +++ b/docs/sql-ref-null-semantics.md @@ -20,6 +20,7 @@ license: | --- ### Description + A table consists of a set of rows and each row contains a set of columns. A column is associated with a data type and represents a specific attribute of an entity (for example, `age` is a column of an @@ -61,7 +62,7 @@ the `age` column and this table will be used in various examples in the sections 700Dan50 -### Comparison operators +### Comparison Operators Apache spark supports the standard comparison operators such as '>', '>=', '=', '<' and '<='. The result of these operators is unknown or `NULL` when one of the operands or both the operands are @@ -114,13 +115,14 @@ one or both operands are `NULL`: ### Examples + {% highlight sql %} -- Normal comparison operators return `NULL` when one of the operand is `NULL`. SELECT 5 > null AS expression_output; +-----------------+ |expression_output| +-----------------+ - |null | + | null| +-----------------+ -- Normal comparison operators return `NULL` when both the operands are `NULL`. @@ -128,7 +130,7 @@ SELECT null = null AS expression_output; +-----------------+ |expression_output| +-----------------+ - |null | + | null| +-----------------+ -- Null-safe equal operator return `False` when one of the operand is `NULL` @@ -136,7 +138,7 @@ SELECT 5 <=> null AS expression_output; +-----------------+ |expression_output| +-----------------+ - |false | + | false| +-----------------+ -- Null-safe equal operator return `True` when one of the operand is `NULL` @@ -144,11 +146,12 @@ SELECT NULL <=> NULL; +-----------------+ |expression_output| +-----------------+ - |true | + | true| +-----------------+ {% endhighlight %} -### Logical operators +### Logical Operators + Spark supports standard logical operators such as `AND`, `OR` and `NOT`. These operators take `Boolean` expressions as the arguments and return a `Boolean` value. @@ -205,13 +208,14 @@ The following tables illustrate the behavior of logical operators when one or bo ### Examples + {% highlight sql %} -- Normal comparison operators return `NULL` when one of the operands is `NULL`. SELECT (true OR null) AS expression_output; +-----------------+ |expression_output| +-----------------+ - |true | + | true| +-----------------+ -- Normal comparison operators return `NULL` when both the operands are `NULL`. @@ -219,7 +223,7 @@ SELECT (null OR false) AS expression_output +-----------------+ |expression_output| +-----------------+ - |null | + | null| +-----------------+ -- Null-safe equal operator returns `False` when one of the operands is `NULL` @@ -227,11 +231,12 @@ SELECT NOT(null) AS expression_output; +-----------------+ |expression_output| +-----------------+ - |null | + | null| +-----------------+ {% endhighlight %} ### Expressions + The comparison operators and logical operators are treated as expressions in Spark. Other than these two kinds of expressions, Spark supports other form of expressions such as function expressions, cast expressions, etc. The expressions @@ -240,35 +245,37 @@ in Spark can be broadly classified as : - Expressions that can process `NULL` value operands - The result of these expressions depends on the expression itself. -#### Null intolerant expressions +#### Null Intolerant Expressions + Null intolerant expressions return `NULL` when one or more arguments of expression are `NULL` and most of the expressions fall in this category. ##### Examples + {% highlight sql %} -SELECT concat('John', null) as expression_output; +SELECT concat('John', null) AS expression_output; +-----------------+ |expression_output| +-----------------+ - |null | + | null| +-----------------+ -SELECT positive(null) as expression_output; +SELECT positive(null) AS expression_output; +-----------------+ |expression_output| +-----------------+ - |null | + | null| +-----------------+ -SELECT to_date(null) as expression_output; +SELECT to_date(null) AS expression_output; +-----------------+ |expression_output| +-----------------+ - |null | + | null| +-----------------+ {% endhighlight %} -#### Expressions that can process null value operands. +#### Expressions That Can Process Null Value Operands This class of expressions are designed to handle `NULL` values. The result of the expressions depends on the expression itself. As an example, function expression `isnull` @@ -287,14 +294,14 @@ returns the first non `NULL` value in its list of operands. However, `coalesce` - ATLEASTNNONNULLS - IN - ##### Examples + {% highlight sql %} SELECT isnull(null) AS expression_output; +-----------------+ |expression_output| +-----------------+ - |true | + | true| +-----------------+ -- Returns the first occurrence of non `NULL` value. @@ -302,7 +309,7 @@ SELECT coalesce(null, null, 3, null) AS expression_output; +-----------------+ |expression_output| +-----------------+ - |3 | + | 3| +-----------------+ -- Returns `NULL` as all its operands are `NULL`. @@ -310,18 +317,19 @@ SELECT coalesce(null, null, null, null) AS expression_output; +-----------------+ |expression_output| +-----------------+ - |null | + | null| +-----------------+ -SELECT isnan(null) as expression_output; +SELECT isnan(null) AS expression_output; +-----------------+ |expression_output| +-----------------+ - |false | + | false| +-----------------+ {% endhighlight %} #### Builtin Aggregate Expressions + Aggregate functions compute a single result by processing a set of input rows. Below are the rules of how `NULL` values are handled by aggregate functions. - `NULL` values are ignored from processing by all the aggregate functions. @@ -337,13 +345,14 @@ the rules of how `NULL` values are handled by aggregate functions. - SOME #### Examples + {% highlight sql %} -- `count(*)` does not skip `NULL` values. SELECT count(*) FROM person; +--------+ |count(1)| +--------+ - |7 | + | 7| +--------+ -- `NULL` values in column `age` are skipped from processing. @@ -351,7 +360,7 @@ SELECT count(age) FROM person; +----------+ |count(age)| +----------+ - |5 | + | 5| +----------+ -- `count(*)` on an empty input set returns 0. This is unlike the other @@ -360,7 +369,7 @@ SELECT count(*) FROM person where 1 = 0; +--------+ |count(1)| +--------+ - |0 | + | 0| +--------+ -- `NULL` values are excluded from computation of maximum value. @@ -368,7 +377,7 @@ SELECT max(age) FROM person; +--------+ |max(age)| +--------+ - |50 | + | 50| +--------+ -- `max` returns `NULL` on an empty input set. @@ -376,44 +385,45 @@ SELECT max(age) FROM person where 1 = 0; +--------+ |max(age)| +--------+ - |null | + | null| +--------+ - {% endhighlight %} -### Condition expressions in WHERE, HAVING and JOIN clauses. +### Condition Expressions in WHERE, HAVING and JOIN Clauses + `WHERE`, `HAVING` operators filter rows based on the user specified condition. A `JOIN` operator is used to combine rows from two tables based on a join condition. For all the three operators, a condition expression is a boolean expression and can return True, False or Unknown (NULL). They are "satisfied" if the result of the condition is `True`. #### Examples + {% highlight sql %} -- Persons whose age is unknown (`NULL`) are filtered out from the result set. SELECT * FROM person WHERE age > 0; +--------+---+ - |name |age| + | name|age| +--------+---+ - |Michelle|30 | - |Fred |50 | - |Mike |18 | - |Dan |50 | - |Joe |30 | + |Michelle| 30| + | Fred| 50| + | Mike| 18| + | Dan| 50| + | Joe| 30| +--------+---+ -- `IS NULL` expression is used in disjunction to select the persons -- with unknown (`NULL`) records. SELECT * FROM person WHERE age > 0 OR age IS NULL; +--------+----+ - |name |age | + | name| age| +--------+----+ - |Albert |null| - |Michelle|30 | - |Fred |50 | - |Mike |18 | - |Dan |50 | - |Marry |null| - |Joe |30 | + | Albert|null| + |Michelle| 30| + | Fred| 50| + | Mike| 18| + | Dan| 50| + | Marry|null| + | Joe| 30| +--------+----+ -- Person with unknown(`NULL`) ages are skipped from processing. @@ -421,135 +431,139 @@ SELECT * FROM person GROUP BY age HAVING max(age) > 18; +---+--------+ |age|count(1)| +---+--------+ - |50 |2 | - |30 |2 | + | 50| 2| + | 30| 2| +---+--------+ -- A self join case with a join condition `p1.age = p2.age AND p1.name = p2.name`. -- The persons with unknown age (`NULL`) are filtered out by the join operator. SELECT * FROM person p1, person p2 -WHERE p1.age = p2.age - AND p1.name = p2.name; + WHERE p1.age = p2.age + AND p1.name = p2.name; +--------+---+--------+---+ - |name |age|name |age| + | name|age| name|age| +--------+---+--------+---+ - |Michelle|30 |Michelle|30 | - |Fred |50 |Fred |50 | - |Mike |18 |Mike |18 | - |Dan |50 |Dan |50 | - |Joe |30 |Joe |30 | + |Michelle| 30|Michelle| 30| + | Fred| 50| Fred| 50| + | Mike| 18| Mike| 18| + | Dan| 50| Dan| 50| + | Joe| 30| Joe| 30| +--------+---+--------+---+ -- The age column from both legs of join are compared using null-safe equal which -- is why the persons with unknown age (`NULL`) are qualified by the join. SELECT * FROM person p1, person p2 -WHERE p1.age <=> p2.age - AND p1.name = p2.name; -+--------+----+--------+----+ -| name| age| name| age| -+--------+----+--------+----+ -| Albert|null| Albert|null| -|Michelle| 30|Michelle| 30| -| Fred| 50| Fred| 50| -| Mike| 18| Mike| 18| -| Dan| 50| Dan| 50| -| Marry|null| Marry|null| -| Joe| 30| Joe| 30| -+--------+----+--------+----+ - + WHERE p1.age <=> p2.age + AND p1.name = p2.name; + +--------+----+--------+----+ + | name| age| name| age| + +--------+----+--------+----+ + | Albert|null| Albert|null| + |Michelle| 30|Michelle| 30| + | Fred| 50| Fred| 50| + | Mike| 18| Mike| 18| + | Dan| 50| Dan| 50| + | Marry|null| Marry|null| + | Joe| 30| Joe| 30| + +--------+----+--------+----+ {% endhighlight %} -### Aggregate operator (GROUP BY, DISTINCT) +### Aggregate Operator (GROUP BY, DISTINCT) + As discussed in the previous section [comparison operator](sql-ref-null-semantics.html#comparison-operators), two `NULL` values are not equal. However, for the purpose of grouping and distinct processing, the two or more values with `NULL data`are grouped together into the same bucket. This behaviour is conformant with SQL standard and with other enterprise database management systems. #### Examples + {% highlight sql %} -- `NULL` values are put in one bucket in `GROUP BY` processing. SELECT age, count(*) FROM person GROUP BY age; +----+--------+ - |age |count(1)| + | age|count(1)| +----+--------+ - |null|2 | - |50 |2 | - |30 |2 | - |18 |1 | + |null| 2| + | 50| 2| + | 30| 2| + | 18| 1| +----+--------+ -- All `NULL` ages are considered one distinct value in `DISTINCT` processing. SELECT DISTINCT age FROM person; +----+ - |age | + | age| +----+ |null| - |50 | - |30 | - |18 | + | 50| + | 30| + | 18| +----+ - {% endhighlight %} -### Sort operator (ORDER BY Clause) +### Sort Operator (ORDER BY Clause) + Spark SQL supports null ordering specification in `ORDER BY` clause. Spark processes the `ORDER BY` clause by placing all the `NULL` values at first or at last depending on the null ordering specification. By default, all the `NULL` values are placed at first. #### Examples + {% highlight sql %} -- `NULL` values are shown at first and other values -- are sorted in ascending way. SELECT age, name FROM person ORDER BY age; +----+--------+ - |age |name | + | age| name| +----+--------+ - |null|Marry | - |null|Albert | - |18 |Mike | - |30 |Michelle| - |30 |Joe | - |50 |Fred | - |50 |Dan | + |null| Marry| + |null| Albert| + | 18| Mike| + | 30|Michelle| + | 30| Joe| + | 50| Fred| + | 50| Dan| +----+--------+ -- Column values other than `NULL` are sorted in ascending -- way and `NULL` values are shown at the last. SELECT age, name FROM person ORDER BY age NULLS LAST; +----+--------+ - |age |name | + | age| name| +----+--------+ - |18 |Mike | - |30 |Michelle| - |30 |Joe | - |50 |Dan | - |50 |Fred | - |null|Marry | - |null|Albert | + | 18| Mike| + | 30|Michelle| + | 30| Joe| + | 50| Dan| + | 50| Fred| + |null| Marry| + |null| Albert| +----+--------+ -- Columns other than `NULL` values are sorted in descending -- and `NULL` values are shown at the last. SELECT age, name FROM person ORDER BY age DESC NULLS LAST; +----+--------+ - |age |name | + | age| name| +----+--------+ - |50 |Fred | - |50 |Dan | - |30 |Michelle| - |30 |Joe | - |18 |Mike | - |null|Marry | - |null|Albert | + | 50| Fred| + | 50| Dan| + | 30|Michelle| + | 30| Joe| + | 18| Mike| + |null| Marry| + |null| Albert| +----+--------+ {% endhighlight %} -### Set operators (UNION, INTERSECT, EXCEPT) +### Set Operators (UNION, INTERSECT, EXCEPT) + `NULL` values are compared in a null-safe manner for equality in the context of set operations. That means when comparing rows, two `NULL` values are considered equal unlike the regular `EqualTo`(`=`) operator. #### Examples + {% highlight sql %} CREATE VIEW unknown_age SELECT * FROM person WHERE age IS NULL; @@ -557,51 +571,51 @@ CREATE VIEW unknown_age SELECT * FROM person WHERE age IS NULL; -- result set. The comparison between columns of the row are done -- in a null-safe manner. SELECT name, age FROM person -INTERSECT -SELECT name, age from unknown_age; + INTERSECT + SELECT name, age from unknown_age; +------+----+ - |name |age | + | name| age| +------+----+ |Albert|null| - |Marry |null| + | Marry|null| +------+----+ -- `NULL` values from two legs of the `EXCEPT` are not in output. -- This basically shows that the comparison happens in a null-safe manner. SELECT age, name FROM person -EXCEPT -SELECT age FROM unknown_age; + EXCEPT + SELECT age FROM unknown_age; +---+--------+ - |age|name | + |age| name| +---+--------+ - |30 |Joe | - |50 |Fred | - |30 |Michelle| - |18 |Mike | - |50 |Dan | + | 30| Joe| + | 50| Fred| + | 30|Michelle| + | 18| Mike| + | 50| Dan| +---+--------+ -- Performs `UNION` operation between two sets of data. -- The comparison between columns of the row ae done in -- null-safe manner. SELECT name, age FROM person -UNION -SELECT name, age FROM unknown_age; + UNION + SELECT name, age FROM unknown_age; +--------+----+ - |name |age | + | name| age| +--------+----+ - |Albert |null| - |Joe |30 | - |Michelle|30 | - |Marry |null| - |Fred |50 | - |Mike |18 | - |Dan |50 | + | Albert|null| + | Joe| 30| + |Michelle| 30| + | Marry|null| + | Fred| 50| + | Mike| 18| + | Dan| 50| +--------+----+ {% endhighlight %} - ### EXISTS/NOT EXISTS Subquery + In Spark, EXISTS and NOT EXISTS expressions are allowed inside a WHERE clause. These are boolean expressions which return either `TRUE` or `FALSE`. In other words, EXISTS is a membership condition and returns `TRUE` @@ -614,20 +628,21 @@ the subquery. They are normally faster because they can be converted to semijoins / anti-semijoins without special provisions for null awareness. #### Examples + {% highlight sql %} -- Even if subquery produces rows with `NULL` values, the `EXISTS` expression -- evaluates to `TRUE` as the subquery produces 1 row. SELECT * FROM person WHERE EXISTS (SELECT null); +--------+----+ - |name |age | + | name| age| +--------+----+ - |Albert |null| - |Michelle|30 | - |Fred |50 | - |Mike |18 | - |Dan |50 | - |Marry |null| - |Joe |30 | + | Albert|null| + |Michelle| 30| + | Fred| 50| + | Mike| 18| + | Dan| 50| + | Marry|null| + | Joe| 30| +--------+----+ -- `NOT EXISTS` expression returns `FALSE`. It returns `TRUE` only when @@ -641,19 +656,20 @@ SELECT * FROM person WHERE NOT EXISTS (SELECT null); -- `NOT EXISTS` expression returns `TRUE`. SELECT * FROM person WHERE NOT EXISTS (SELECT 1 WHERE 1 = 0); +--------+----+ - |name |age | + | name| age| +--------+----+ - |Albert |null| - |Michelle|30 | - |Fred |50 | - |Mike |18 | - |Dan |50 | - |Marry |null| - |Joe |30 | + | Albert|null| + |Michelle| 30| + | Fred| 50| + | Mike| 18| + | Dan| 50| + | Marry|null| + | Joe| 30| +--------+----+ {% endhighlight %} ### IN/NOT IN Subquery + In Spark, `IN` and `NOT IN` expressions are allowed inside a WHERE clause of a query. Unlike the `EXISTS` expression, `IN` expression can return a `TRUE`, `FALSE` or `UNKNOWN (NULL)` value. Conceptually a `IN` expression is semantically @@ -675,6 +691,7 @@ This is because IN returns UNKNOWN if the value is not in the list containing `N and because NOT UNKNOWN is again UNKNOWN. #### Examples + {% highlight sql %} -- The subquery has only `NULL` value in its result set. Therefore, -- the result of `IN` predicate is UNKNOWN. @@ -687,22 +704,21 @@ SELECT * FROM person WHERE age IN (SELECT null); -- The subquery has `NULL` value in the result set as well as a valid -- value `50`. Rows with age = 50 are returned. SELECT * FROM person -WHERE age IN (SELECT age FROM VALUES (50), (null) sub(age)); + WHERE age IN (SELECT age FROM VALUES (50), (null) sub(age)); +----+---+ |name|age| +----+---+ - |Fred|50 | - |Dan |50 | + |Fred| 50| + | Dan| 50| +----+---+ -- Since subquery has `NULL` value in the result set, the `NOT IN` -- predicate would return UNKNOWN. Hence, no rows are -- qualified for this query. SELECT * FROM person -WHERE age NOT IN (SELECT age FROM VALUES (50), (null) sub(age)); + WHERE age NOT IN (SELECT age FROM VALUES (50), (null) sub(age)); +----+---+ |name|age| +----+---+ +----+---+ - {% endhighlight %} diff --git a/docs/sql-ref-syntax-aux-analyze-table.md b/docs/sql-ref-syntax-aux-analyze-table.md index 40513e836b026..739e692680233 100644 --- a/docs/sql-ref-syntax-aux-analyze-table.md +++ b/docs/sql-ref-syntax-aux-analyze-table.md @@ -24,13 +24,14 @@ license: | The `ANALYZE TABLE` statement collects statistics about the table to be used by the query optimizer to find a better query execution plan. ### Syntax + {% highlight sql %} ANALYZE TABLE table_identifier [ partition_spec ] COMPUTE STATISTICS [ NOSCAN | FOR COLUMNS col [ , ... ] | FOR ALL COLUMNS ] - {% endhighlight %} ### Parameters +
    table_identifier
    @@ -69,41 +70,69 @@ ANALYZE TABLE table_identifier [ partition_spec ]
    ### Examples -{% highlight sql %} - ANALYZE TABLE students COMPUTE STATISTICS NOSCAN; - - DESC EXTENDED students; - ...... - Statistics 2820 bytes - ...... - - ANALYZE TABLE students COMPUTE STATISTICS; - - DESC EXTENDED students; - ...... - Statistics 2820 bytes, 3 rows - ...... - - ANALYZE TABLE students PARTITION (student_id = 111111) COMPUTE STATISTICS; - - DESC EXTENDED students PARTITION (student_id = 111111); - ...... - Partition Statistics 919 bytes, 1 rows - ...... - - ANALYZE TABLE students COMPUTE STATISTICS FOR COLUMNS name; - - DESC EXTENDED students name; - =default tbl=students - col_name name - data_type string - comment NULL - min NULL - max NULL - num_nulls 0 - distinct_count 3 - avg_col_len 11 - max_col_len 13 - histogram NULL +{% highlight sql %} +CREATE TABLE students (name STRING, student_id INT) PARTITIONED BY (student_id); +INSERT INTO students PARTITION (student_id = 111111) VALUES ('Mark'); +INSERT INTO students PARTITION (student_id = 222222) VALUES ('John'); + +ANALYZE TABLE students COMPUTE STATISTICS NOSCAN; + +DESC EXTENDED students; + +--------------------+--------------------+-------+ + | col_name| data_type|comment| + +--------------------+--------------------+-------+ + | name| string| null| + | student_id| int| null| + | ...| ...| ...| + | Statistics| 864 bytes| | + | ...| ...| ...| + | Partition Provider| Catalog| | + +--------------------+--------------------+-------+ + +ANALYZE TABLE students COMPUTE STATISTICS; + +DESC EXTENDED students; + +--------------------+--------------------+-------+ + | col_name| data_type|comment| + +--------------------+--------------------+-------+ + | name| string| null| + | student_id| int| null| + | ...| ...| ...| + | Statistics| 864 bytes, 2 rows| | + | ...| ...| ...| + | Partition Provider| Catalog| | + +--------------------+--------------------+-------+ + +ANALYZE TABLE students PARTITION (student_id = 111111) COMPUTE STATISTICS; + +DESC EXTENDED students PARTITION (student_id = 111111); + +--------------------+--------------------+-------+ + | col_name| data_type|comment| + +--------------------+--------------------+-------+ + | name| string| null| + | student_id| int| null| + | ...| ...| ...| + |Partition Statistics| 432 bytes, 1 rows| | + | ...| ...| ...| + | OutputFormat|org.apache.hadoop...| | + +--------------------+--------------------+-------+ + +ANALYZE TABLE students COMPUTE STATISTICS FOR COLUMNS name; + +DESC EXTENDED students name; + +--------------+----------+ + | info_name|info_value| + +--------------+----------+ + | col_name| name| + | data_type| string| + | comment| NULL| + | min| NULL| + | max| NULL| + | num_nulls| 0| + |distinct_count| 2| + | avg_col_len| 4| + | max_col_len| 4| + | histogram| NULL| + +--------------+----------+ {% endhighlight %} diff --git a/docs/sql-ref-syntax-aux-analyze.md b/docs/sql-ref-syntax-aux-analyze.md index b1bdc73657724..4c68e6b9ec974 100644 --- a/docs/sql-ref-syntax-aux-analyze.md +++ b/docs/sql-ref-syntax-aux-analyze.md @@ -19,4 +19,4 @@ license: | limitations under the License. --- -* [ANALYZE TABLE statement](sql-ref-syntax-aux-analyze-table.html) + * [ANALYZE TABLE statement](sql-ref-syntax-aux-analyze-table.html) diff --git a/docs/sql-ref-syntax-aux-cache-cache-table.md b/docs/sql-ref-syntax-aux-cache-cache-table.md index 20ade1961ab0b..11f682cc10891 100644 --- a/docs/sql-ref-syntax-aux-cache-cache-table.md +++ b/docs/sql-ref-syntax-aux-cache-cache-table.md @@ -20,16 +20,19 @@ license: | --- ### Description + `CACHE TABLE` statement caches contents of a table or output of a query with the given storage level. If a query is cached, then a temp view will be created for this query. This reduces scanning of the original files in future queries. ### Syntax + {% highlight sql %} CACHE [ LAZY ] TABLE table_identifier [ OPTIONS ( 'storageLevel' [ = ] value ) ] [ [ AS ] query ] {% endhighlight %} ### Parameters +
    LAZY
    Only cache the table when it is first used, instead of immediately.
    @@ -80,12 +83,14 @@ CACHE [ LAZY ] TABLE table_identifier
    ### Examples + {% highlight sql %} CACHE TABLE testCache OPTIONS ('storageLevel' 'DISK_ONLY') SELECT * FROM testData; {% endhighlight %} ### Related Statements - * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) - * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) - * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) + * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) + * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) + * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) + * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-cache-clear-cache.md b/docs/sql-ref-syntax-aux-cache-clear-cache.md index 57ba77d59ea7b..47889691148b7 100644 --- a/docs/sql-ref-syntax-aux-cache-clear-cache.md +++ b/docs/sql-ref-syntax-aux-cache-clear-cache.md @@ -20,20 +20,24 @@ license: | --- ### Description + `CLEAR CACHE` removes the entries and associated data from the in-memory and/or on-disk cache for all cached tables and views. ### Syntax + {% highlight sql %} CLEAR CACHE {% endhighlight %} ### Examples + {% highlight sql %} CLEAR CACHE; {% endhighlight %} ### Related Statements + * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) - + * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-cache-refresh.md b/docs/sql-ref-syntax-aux-cache-refresh.md index 4c56893aeca98..25f7ede1d324e 100644 --- a/docs/sql-ref-syntax-aux-cache-refresh.md +++ b/docs/sql-ref-syntax-aux-cache-refresh.md @@ -20,35 +20,39 @@ license: | --- ### Description + `REFRESH` is used to invalidate and refresh all the cached data (and the associated metadata) for all Datasets that contains the given data source path. Path matching is by prefix, i.e. "/" would invalidate everything that is cached. ### Syntax + {% highlight sql %} REFRESH resource_path {% endhighlight %} ### Parameters +
    resource_path
    The path of the resource that is to be refreshed.
    ### Examples + {% highlight sql %} - -- The Path is resolved using the datasource's File Index. +-- The Path is resolved using the datasource's File Index. CREATE TABLE test(ID INT) using parquet; INSERT INTO test SELECT 1000; CACHE TABLE test; INSERT INTO test SELECT 100; REFRESH "hdfs://path/to/table"; - {% endhighlight %} ### Related Statements -- [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) -- [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) -- [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) -- [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) + + * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) + * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) + * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) + * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md index 69e21c258a333..95fd91c3c4807 100644 --- a/docs/sql-ref-syntax-aux-cache-uncache-table.md +++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md @@ -20,15 +20,18 @@ license: | --- ### Description + `UNCACHE TABLE` removes the entries and associated data from the in-memory and/or on-disk cache for a given table or view. The underlying entries should already have been brought to cache by previous `CACHE TABLE` operation. `UNCACHE TABLE` on a non-existent table throws an exception if `IF EXISTS` is not specified. ### Syntax + {% highlight sql %} UNCACHE TABLE [ IF EXISTS ] table_identifier {% endhighlight %} ### Parameters +
    table_identifier
    @@ -41,11 +44,14 @@ UNCACHE TABLE [ IF EXISTS ] table_identifier
    ### Examples + {% highlight sql %} UNCACHE TABLE t1; {% endhighlight %} ### Related Statements + * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) + * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-cache.md b/docs/sql-ref-syntax-aux-cache.md index e4a640a6edbe5..418b8cc3403b5 100644 --- a/docs/sql-ref-syntax-aux-cache.md +++ b/docs/sql-ref-syntax-aux-cache.md @@ -19,7 +19,8 @@ license: | limitations under the License. --- -* [CACHE TABLE statement](sql-ref-syntax-aux-cache-cache-table.html) -* [UNCACHE TABLE statement](sql-ref-syntax-aux-cache-uncache-table.html) -* [CLEAR CACHE statement](sql-ref-syntax-aux-cache-clear-cache.html) -* [REFRESH TABLE statement](sql-ref-syntax-aux-refresh-table.html) + * [CACHE TABLE statement](sql-ref-syntax-aux-cache-cache-table.html) + * [UNCACHE TABLE statement](sql-ref-syntax-aux-cache-uncache-table.html) + * [CLEAR CACHE statement](sql-ref-syntax-aux-cache-clear-cache.html) + * [REFRESH TABLE statement](sql-ref-syntax-aux-refresh-table.html) + * [REFRESH statement](sql-ref-syntax-aux-cache-refresh.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index 8ee61514ee4ef..e7e6dda4e25ee 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -20,19 +20,22 @@ license: | --- ### Description -Reset all the properties specific to the current session to their default values. After RESET command, executing SET command will output empty. + +Reset any runtime configurations specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. ### Syntax + {% highlight sql %} RESET {% endhighlight %} - ### Examples + {% highlight sql %} --- Reset all the properties specific to the current session to their default values. +-- Reset any runtime configurations specific to the current session which were set via the SET command to their default values. RESET; {% endhighlight %} ### Related Statements -- [SET](sql-ref-syntax-aux-conf-mgmt-set.html) + + * [SET](sql-ref-syntax-aux-conf-mgmt-set.html) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-set.md b/docs/sql-ref-syntax-aux-conf-mgmt-set.md index f05dde3f567ee..2ca51307c3aae 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-set.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-set.md @@ -20,9 +20,11 @@ license: | --- ### Description + The SET command sets a property, returns the value of an existing property or returns all SQLConf properties with value and meaning. ### Syntax + {% highlight sql %} SET SET [ -v ] @@ -30,6 +32,7 @@ SET property_key[ = property_value ] {% endhighlight %} ### Parameters +
    -v
    Outputs the key, value and meaning of existing SQLConf properties.
    @@ -46,9 +49,10 @@ SET property_key[ = property_value ]
    ### Examples + {% highlight sql %} -- Set a property. -SET spark.sql.variable.substitute=false; +SET spark.sql.variable.substitute=false; -- List all SQLConf properties with value and meaning. SET -v; @@ -57,13 +61,14 @@ SET -v; SET; -- List the value of specified property key. -SET spark.sql.variable.substitute; - +--------------------------------+--------+ - | key | value | - +--------------------------------+--------+ - | spark.sql.variable.substitute | false | - +--------------------------------+--------+ +SET spark.sql.variable.substitute; + +-----------------------------+-----+ + | key|value| + +-----------------------------+-----+ + |spark.sql.variable.substitute|false| + +-----------------------------+-----+ {% endhighlight %} ### Related Statements -- [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) + + * [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt.md b/docs/sql-ref-syntax-aux-conf-mgmt.md index 7c5d9cc895c10..f5e48ef2fee30 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt.md @@ -19,5 +19,5 @@ license: | limitations under the License. --- -* [SET](sql-ref-syntax-aux-conf-mgmt-set.html) -* [UNSET](sql-ref-syntax-aux-conf-mgmt-reset.html) + * [SET](sql-ref-syntax-aux-conf-mgmt-set.html) + * [UNSET](sql-ref-syntax-aux-conf-mgmt-reset.html) diff --git a/docs/sql-ref-syntax-aux-describe-database.md b/docs/sql-ref-syntax-aux-describe-database.md index 05a64ab2060b4..2f7b1ce984d3e 100644 --- a/docs/sql-ref-syntax-aux-describe-database.md +++ b/docs/sql-ref-syntax-aux-describe-database.md @@ -18,6 +18,7 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description ​ `DESCRIBE DATABASE` statement returns the metadata of an existing database. The metadata information includes database @@ -26,11 +27,13 @@ returns the basic metadata information along with the database properties. The ` interchangeable. ### Syntax + {% highlight sql %} { DESC | DESCRIBE } DATABASE [ EXTENDED ] db_name {% endhighlight %} ### Parameters +
    db_name
    @@ -40,6 +43,7 @@ interchangeable.
    ### Example + {% highlight sql %} -- Create employees DATABASE CREATE DATABASE employees COMMENT 'For software companies'; @@ -49,11 +53,11 @@ CREATE DATABASE employees COMMENT 'For software companies'; -- for the employees DATABASE. DESCRIBE DATABASE employees; +-------------------------+-----------------------------+ - |database_description_item|database_description_value | + |database_description_item| database_description_value| +-------------------------+-----------------------------+ - |Database Name |employees | - |Description |For software companies | - |Location |file:/Users/Temp/employees.db| + | Database Name| employees| + | Description| For software companies| + | Location|file:/Users/Temp/employees.db| +-------------------------+-----------------------------+ -- Create employees DATABASE @@ -65,12 +69,12 @@ ALTER DATABASE employees SET DBPROPERTIES ('Create-by' = 'Kevin', 'Create-date' -- Describe employees DATABASE with EXTENDED option to return additional database properties DESCRIBE DATABASE EXTENDED employees; +-------------------------+---------------------------------------------+ - |database_description_item|database_description_value | + |database_description_item| database_description_value| +-------------------------+---------------------------------------------+ - |Database Name |employees | - |Description |For software companies | - |Location |file:/Users/Temp/employees.db | - |Properties |((Create-by,kevin), (Create-date,09/01/2019))| + | Database Name| employees| + | Description| For software companies| + | Location| file:/Users/Temp/employees.db| + | Properties|((Create-by,kevin), (Create-date,09/01/2019))| +-------------------------+---------------------------------------------+ -- Create deployment SCHEMA @@ -81,14 +85,14 @@ DESC DATABASE deployment; +-------------------------+------------------------------+ |database_description_item|database_description_value | +-------------------------+------------------------------+ - |Database Name |deployment | - |Description |Deployment environment | - |Location |file:/Users/Temp/deployment.db| + | Database Name| deployment| + | Description| Deployment environment| + | Location|file:/Users/Temp/deployment.db| +-------------------------+------------------------------+ - {% endhighlight %} ### Related Statements -- [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) -- [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) -- [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) + + * [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) + * [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) + * [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) diff --git a/docs/sql-ref-syntax-aux-describe-function.md b/docs/sql-ref-syntax-aux-describe-function.md index f3c9c625b97b8..a4ff76bddf782 100644 --- a/docs/sql-ref-syntax-aux-describe-function.md +++ b/docs/sql-ref-syntax-aux-describe-function.md @@ -18,6 +18,7 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description `DESCRIBE FUNCTION` statement returns the basic metadata information of an @@ -26,11 +27,13 @@ class and the usage details. If the optional `EXTENDED` option is specified, th metadata information is returned along with the extended usage information. ### Syntax + {% highlight sql %} { DESC | DESCRIBE } FUNCTION [ EXTENDED ] function_name {% endhighlight %} ### Parameters +
    function_name
    @@ -46,6 +49,7 @@ metadata information is returned along with the extended usage information.
    ### Examples + {% highlight sql %} -- Describe a builtin scalar function. -- Returns function name, implementing class and usage @@ -107,6 +111,7 @@ DESC FUNCTION EXTENDED explode {% endhighlight %} ### Related Statements -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) -- [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) -- [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) + + * [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) + * [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) + * [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) diff --git a/docs/sql-ref-syntax-aux-describe-query.md b/docs/sql-ref-syntax-aux-describe-query.md index b07ebe78193d1..f64416adc556d 100644 --- a/docs/sql-ref-syntax-aux-describe-query.md +++ b/docs/sql-ref-syntax-aux-describe-query.md @@ -20,16 +20,19 @@ license: | --- ### Description + The `DESCRIBE QUERY` statement is used to return the metadata of output of a query. A shorthand `DESC` may be used instead of `DESCRIBE` to describe the query output. ### Syntax + {% highlight sql %} { DESC | DESCRIBE } [ QUERY ] input_statement {% endhighlight %} ### Parameters +
    QUERY
    This clause is optional and may be omitted.
    @@ -49,6 +52,7 @@ describe the query output.
    ### Examples + {% highlight sql %} -- Create table `person` CREATE TABLE person (name STRING , age INT COMMENT 'Age column', address STRING); @@ -56,19 +60,19 @@ CREATE TABLE person (name STRING , age INT COMMENT 'Age column', address STRING) -- Returns column metadata information for a simple select query DESCRIBE QUERY select age, sum(age) FROM person GROUP BY age; +--------+---------+----------+ - |col_name|data_type|comment | + |col_name|data_type| comment| +--------+---------+----------+ - |age |int |Age column| - |sum(age)|bigint |null | + | age| int|Age column| + |sum(age)| bigint| null| +--------+---------+----------+ -- Returns column metadata information for common table expression (`CTE`). DESCRIBE QUERY WITH all_names_cte - AS (SELECT name from person) SELECT * FROM all_names_cte; + AS (SELECT name from person) SELECT * FROM all_names_cte; +--------+---------+-------+ |col_name|data_type|comment| +--------+---------+-------+ - |name |string |null | + | name| string| null| +--------+---------+-------+ -- Returns column metadata information for a inline table. @@ -76,32 +80,33 @@ DESC QUERY VALUES(100, 'John', 10000.20D) AS employee(id, name, salary); +--------+---------+-------+ |col_name|data_type|comment| +--------+---------+-------+ - |id |int |null | - |name |string |null | - |salary |double |null | + | id| int| null| + | name| string| null| + | salary| double| null| +--------+---------+-------+ -- Returns column metadata information for `TABLE` statement. DESC QUERY TABLE person; +--------+---------+----------+ - |col_name|data_type|comment | + |col_name|data_type| comment| +--------+---------+----------+ - |name |string |null | - |age |int |Age column| - |address |string |null | + | name| string| null| + | age| int| Agecolumn| + | address| string| null| +--------+---------+----------+ -- Returns column metadata information for a `FROM` statement. -- `QUERY` clause is optional and can be omitted. DESCRIBE FROM person SELECT age; +--------+---------+----------+ - |col_name|data_type|comment | + |col_name|data_type| comment| +--------+---------+----------+ - |age |int |Age column| + | age| int| Agecolumn| +--------+---------+----------+ {% endhighlight %} ### Related Statements -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) -- [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) -- [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) + + * [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) + * [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) + * [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) diff --git a/docs/sql-ref-syntax-aux-describe-table.md b/docs/sql-ref-syntax-aux-describe-table.md index 4e6aeb5b6f349..a8eee97b4dc1e 100644 --- a/docs/sql-ref-syntax-aux-describe-table.md +++ b/docs/sql-ref-syntax-aux-describe-table.md @@ -18,18 +18,22 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description + `DESCRIBE TABLE` statement returns the basic metadata information of a table. The metadata information includes column name, column type and column comment. Optionally a partition spec or column name may be specified to return the metadata pertaining to a partition or column respectively. ### Syntax + {% highlight sql %} { DESC | DESCRIBE } [ TABLE ] [ format ] table_identifier [ partition_spec ] [ col_name ] {% endhighlight %} ### Parameters +
    format
    @@ -69,101 +73,105 @@ to return the metadata pertaining to a partition or column respectively.
    ### Examples + {% highlight sql %} -- Creates a table `customer`. Assumes current database is `salesdb`. CREATE TABLE customer( - cust_id INT, - state VARCHAR(20), - name STRING COMMENT 'Short name' - ) - USING parquet - PARTITION BY state; - ; - + cust_id INT, + state VARCHAR(20), + name STRING COMMENT 'Short name' + ) + USING parquet + PARTITIONED BY (state); + +INSERT INTO customer PARTITION (state = 'AR') VALUES (100, 'Mike'); + -- Returns basic metadata information for unqualified table `customer` DESCRIBE TABLE customer; +-----------------------+---------+----------+ - |col_name |data_type|comment | + | col_name|data_type| comment| +-----------------------+---------+----------+ - |cust_id |int |null | - |name |string |Short name| - |state |string |null | + | cust_id| int| null| + | name| string|Short name| + | state| string| null| |# Partition Information| | | - |# col_name |data_type|comment | - |state |string |null | + | # col_name|data_type| comment| + | state| string| null| +-----------------------+---------+----------+ -- Returns basic metadata information for qualified table `customer` DESCRIBE TABLE salesdb.customer; +-----------------------+---------+----------+ - |col_name |data_type|comment | + | col_name|data_type| comment| +-----------------------+---------+----------+ - |cust_id |int |null | - |name |string |Short name| - |state |string |null | + | cust_id| int| null| + | name| string|Short name| + | state| string| null| |# Partition Information| | | - |# col_name |data_type|comment | - |state |string |null | + | # col_name|data_type| comment| + | state| string| null| +-----------------------+---------+----------+ -- Returns additional metadata such as parent database, owner, access time etc. DESCRIBE TABLE EXTENDED customer; +----------------------------+------------------------------+----------+ - |col_name |data_type |comment | + | col_name| data_type| comment| +----------------------------+------------------------------+----------+ - |cust_id |int |null | - |name |string |Short name| - |state |string |null | - |# Partition Information | | | - |# col_name |data_type |comment | - |state |string |null | + | cust_id| int| null| + | name| string|Short name| + | state| string| null| + | # Partition Information| | | + | # col_name| data_type| comment| + | state| string| null| | | | | |# Detailed Table Information| | | - |Database |salesdb | | - |Table |customer | | - |Owner | | | - |Created Time |Fri Aug 30 09:26:04 PDT 2019 | | - |Last Access |Wed Dec 31 16:00:00 PST 1969 | | - |Created By | | | - |Type |MANAGED | | - |Provider |parquet | | - |Location |file:.../salesdb.db/customer | | - |Serde Library |...serde.ParquetHiveSerDe | | - |InputFormat |...MapredParquetInputFormat | | - |OutputFormat |...MapredParquetOutputFormat | | + | Database| default| | + | Table| customer| | + | Owner|
    | | + | Created Time| Tue Apr 07 22:56:34 JST 2020| | + | Last Access| UNKNOWN| | + | Created By| | | + | Type| MANAGED| | + | Provider| parquet| | + | Location|file:/tmp/salesdb.db/custom...| | + | Serde Library|org.apache.hadoop.hive.ql.i...| | + | InputFormat|org.apache.hadoop.hive.ql.i...| | + | OutputFormat|org.apache.hadoop.hive.ql.i...| | + | Partition Provider| Catalog| | +----------------------------+------------------------------+----------+ -- Returns partition metadata such as partitioning column name, column type and comment. -DESCRIBE TABLE customer PARTITION (state = 'AR'); - - +--------------------------------+-----------------------------------------+----------+ - |col_name |data_type |comment | - +--------------------------------+-----------------------------------------+----------+ - |cust_id |int |null | - |name |string |Short name| - |state |string |null | - |# Partition Information | | | - |# col_name |data_type |comment | - |state |string |null | - | | | | - |# Detailed Partition Information| | | - |Database |salesdb | | - |Table |customer | | - |Partition Values |[state=AR] | | - |Location |file:.../salesdb.db/customer/state=AR | | - |Serde Library |...serde.ParquetHiveSerDe | | - |InputFormat |...parquet.MapredParquetInputFormat | | - |OutputFormat |...parquet.MapredParquetOutputFormat | | - |Storage Properties |[path=file:.../salesdb.db/customer, | | - | | serialization.format=1] | | - |Partition Parameters |{rawDataSize=-1, numFiles=1l, | | - | | transient_lastDdlTime=1567185245, | | - | | totalSize=688, | | - | | COLUMN_STATS_ACCURATE=false, numRows=-1}| | - |Created Time |Fri Aug 30 10:14:05 PDT 2019 | | - |Last Access |Wed Dec 31 16:00:00 PST 1969 | | - |Partition Statistics |688 bytes | | - +--------------------------------+-----------------------------------------+----------+ +DESCRIBE TABLE EXTENDED customer PARTITION (state = 'AR'); + +------------------------------+------------------------------+----------+ + | col_name| data_type| comment| + +------------------------------+------------------------------+----------+ + | cust_id| int| null| + | name| string|Short name| + | state| string| null| + | # Partition Information| | | + | # col_name| data_type| comment| + | state| string| null| + | | | | + |# Detailed Partition Inform...| | | + | Database| default| | + | Table| customer| | + | Partition Values| [state=AR]| | + | Location|file:/tmp/salesdb.db/custom...| | + | Serde Library|org.apache.hadoop.hive.ql.i...| | + | InputFormat|org.apache.hadoop.hive.ql.i...| | + | OutputFormat|org.apache.hadoop.hive.ql.i...| | + | Storage Properties|[serialization.format=1, pa...| | + | Partition Parameters|{transient_lastDdlTime=1586...| | + | Created Time| Tue Apr 07 23:05:43 JST 2020| | + | Last Access| UNKNOWN| | + | Partition Statistics| 659 bytes| | + | | | | + | # Storage Information| | | + | Location|file:/tmp/salesdb.db/custom...| | + | Serde Library|org.apache.hadoop.hive.ql.i...| | + | InputFormat|org.apache.hadoop.hive.ql.i...| | + | OutputFormat|org.apache.hadoop.hive.ql.i...| | + +------------------------------+------------------------------+----------+ -- Returns the metadata for `name` column. -- Optional `TABLE` clause is omitted and column is fully qualified. @@ -171,13 +179,14 @@ DESCRIBE customer salesdb.customer.name; +---------+----------+ |info_name|info_value| +---------+----------+ - |col_name |name | - |data_type|string | - |comment |Short name| + | col_name| name| + |data_type| string| + | comment|Short name| +---------+----------+ {% endhighlight %} ### Related Statements -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) -- [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) -- [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) + + * [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) + * [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) + * [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) diff --git a/docs/sql-ref-syntax-aux-describe.md b/docs/sql-ref-syntax-aux-describe.md index 9f17746316480..723943f97aa07 100644 --- a/docs/sql-ref-syntax-aux-describe.md +++ b/docs/sql-ref-syntax-aux-describe.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -* [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) -* [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) -* [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) -* [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) + * [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) + * [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) + * [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) + * [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) diff --git a/docs/sql-ref-syntax-aux-refresh-table.md b/docs/sql-ref-syntax-aux-refresh-table.md index 96d698a1d68b0..165ca68309f4a 100644 --- a/docs/sql-ref-syntax-aux-refresh-table.md +++ b/docs/sql-ref-syntax-aux-refresh-table.md @@ -20,16 +20,19 @@ license: | --- ### Description + `REFRESH TABLE` statement invalidates the cached entries, which include data and metadata of the given table or view. The invalidated cache is populated in lazy manner when the cached table or the query associated with it is executed again. ### Syntax + {% highlight sql %} REFRESH [TABLE] table_identifier {% endhighlight %} ### Parameters +
    table_identifier
    @@ -42,6 +45,7 @@ REFRESH [TABLE] table_identifier
    ### Examples + {% highlight sql %} -- The cached entries of the table will be refreshed -- The table is resolved from the current database as the table name is unqualified. @@ -53,6 +57,8 @@ REFRESH TABLE tempDB.view1; {% endhighlight %} ### Related Statements -- [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) -- [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) -- [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) + + * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) + * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) + * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) + * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md index 7e485cbafe709..0028884308890 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md @@ -20,20 +20,24 @@ license: | --- ### Description + `ADD FILE` can be used to add a single file as well as a directory to the list of resources. The added resource can be listed using [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html). ### Syntax + {% highlight sql %} ADD FILE resource_name {% endhighlight %} ### Parameters +
    resource_name
    The name of the file or directory to be added.
    ### Examples + {% highlight sql %} ADD FILE /tmp/test; ADD FILE "/path/to/file/abc.txt"; @@ -43,6 +47,7 @@ ADD FILE "/path/to/some/directory"; {% endhighlight %} ### Related Statements + * [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) * [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index db0a85013321d..c4020347c1be0 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -20,20 +20,24 @@ license: | --- ### Description + `ADD JAR` adds a JAR file to the list of resources. The added JAR file can be listed using [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html). ### Syntax + {% highlight sql %} ADD JAR file_name {% endhighlight %} ### Parameters +
    file_name
    The name of the JAR file to be added. It could be either on a local file system or a distributed file system.
    ### Examples + {% highlight sql %} ADD JAR /tmp/test.jar; ADD JAR "/path/to/some.jar"; @@ -42,6 +46,7 @@ ADD JAR "/path with space/abc.jar"; {% endhighlight %} ### Related Statements + * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) * [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) * [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md b/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md index c42bf7ae8dd41..eec98e1fbffb5 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md @@ -20,14 +20,17 @@ license: | --- ### Description + `LIST FILE` lists the resources added by [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html). ### Syntax + {% highlight sql %} LIST FILE {% endhighlight %} ### Examples + {% highlight sql %} ADD FILE /tmp/test; ADD FILE /tmp/test_2; @@ -42,6 +45,7 @@ file:/private/tmp/test {% endhighlight %} ### Related Statements + * [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) * [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md index 9d1739753099e..dca4252c90ef2 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md @@ -20,14 +20,17 @@ license: | --- ### Description + `LIST JAR` lists the JARs added by [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html). ### Syntax + {% highlight sql %} LIST JAR {% endhighlight %} ### Examples + {% highlight sql %} ADD JAR /tmp/test.jar; ADD JAR /tmp/test_2.jar; @@ -42,6 +45,7 @@ spark://192.168.1.112:62859/jars/test.jar {% endhighlight %} ### Related Statements + * [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) * [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) * [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt.md b/docs/sql-ref-syntax-aux-resource-mgmt.md index 0885f56bdb7cf..50c12ef7c2beb 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -* [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) -* [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) -* [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) -* [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) + * [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) + * [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) + * [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) + * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) diff --git a/docs/sql-ref-syntax-aux-show-columns.md b/docs/sql-ref-syntax-aux-show-columns.md index 0c8aba83a8403..8f73aac0e3a61 100644 --- a/docs/sql-ref-syntax-aux-show-columns.md +++ b/docs/sql-ref-syntax-aux-show-columns.md @@ -18,15 +18,19 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description + Return the list of columns in a table. If the table does not exist, an exception is thrown. ### Syntax + {% highlight sql %} SHOW COLUMNS table_identifier [ database ] {% endhighlight %} ### Parameters +
    table_identifier
    @@ -54,44 +58,47 @@ SHOW COLUMNS table_identifier [ database ]
    ### Examples + {% highlight sql %} -- Create `customer` table in `salesdb` database; USE salesdb; -CREATE TABLE customer(cust_cd INT, - name VARCHAR(100), - cust_addr STRING); +CREATE TABLE customer( + cust_cd INT, + name VARCHAR(100), + cust_addr STRING); -- List the columns of `customer` table in current database. SHOW COLUMNS IN customer; +---------+ - |col_name | + | col_name| +---------+ - |cust_cd | - |name | + | cust_cd| + | name| |cust_addr| +---------+ -- List the columns of `customer` table in `salesdb` database. SHOW COLUMNS IN salesdb.customer; +---------+ - |col_name | + | col_name| +---------+ - |cust_cd | - |name | + | cust_cd| + | name| |cust_addr| +---------+ -- List the columns of `customer` table in `salesdb` database SHOW COLUMNS IN customer IN salesdb; +---------+ - |col_name | + | col_name| +---------+ - |cust_cd | - |name | + | cust_cd| + | name| |cust_addr| +---------+ {% endhighlight %} ### Related Statements -- [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) -- [SHOW TABLE](sql-ref-syntax-aux-show-table.html) + + * [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) + * [SHOW TABLE](sql-ref-syntax-aux-show-table.html) diff --git a/docs/sql-ref-syntax-aux-show-create-table.md b/docs/sql-ref-syntax-aux-show-create-table.md index 24aba602ab3cf..0a37c96bfc5ab 100644 --- a/docs/sql-ref-syntax-aux-show-create-table.md +++ b/docs/sql-ref-syntax-aux-show-create-table.md @@ -20,14 +20,17 @@ license: | --- ### Description + `SHOW CREATE TABLE` returns the [CREATE TABLE statement](sql-ref-syntax-ddl-create-table.html) or [CREATE VIEW statement](sql-ref-syntax-ddl-create-view.html) that was used to create a given table or view. `SHOW CREATE TABLE` on a non-existent table or a temporary view throws an exception. ### Syntax + {% highlight sql %} SHOW CREATE TABLE table_identifier {% endhighlight %} ### Parameters +
    table_identifier
    @@ -40,31 +43,26 @@ SHOW CREATE TABLE table_identifier
    ### Examples + {% highlight sql %} CREATE TABLE test (c INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' STORED AS TEXTFILE TBLPROPERTIES ('prop1' = 'value1', 'prop2' = 'value2'); -show create table test; - --- the result of SHOW CREATE TABLE test -CREATE TABLE `test`(`c` INT) -ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' -WITH SERDEPROPERTIES ( - 'field.delim' = ',', - 'serialization.format' = ',' -) -STORED AS - INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' - OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' -TBLPROPERTIES ( - 'transient_lastDdlTime' = '1569350233', - 'prop1' = 'value1', - 'prop2' = 'value2' -) - +SHOW CREATE TABLE test; + +----------------------------------------------------+ + | createtab_stmt| + +----------------------------------------------------+ + |CREATE TABLE `default`.`test` (`c` INT) + USING text + TBLPROPERTIES ( + 'transient_lastDdlTime' = '1586269021', + 'prop1' = 'value1', + 'prop2' = 'value2') + +----------------------------------------------------+ {% endhighlight %} ### Related Statements + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) * [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) diff --git a/docs/sql-ref-syntax-aux-show-databases.md b/docs/sql-ref-syntax-aux-show-databases.md index 9d4be21daeabc..0ed34527bcbf3 100644 --- a/docs/sql-ref-syntax-aux-show-databases.md +++ b/docs/sql-ref-syntax-aux-show-databases.md @@ -20,17 +20,20 @@ license: | --- ### Description + Lists the databases that match an optionally supplied string pattern. If no pattern is supplied then the command lists all the databases in the system. Please note that the usage of `SCHEMAS` and `DATABASES` are interchangeable and mean the same thing. ### Syntax + {% highlight sql %} SHOW { DATABASES | SCHEMAS } [ LIKE string_pattern ] {% endhighlight %} ### Parameters +
    LIKE string_pattern
    @@ -40,6 +43,7 @@ SHOW { DATABASES | SCHEMAS } [ LIKE string_pattern ]
    ### Examples + {% highlight sql %} -- Create database. Assumes a database named `default` already exists in -- the system. @@ -55,6 +59,7 @@ SHOW DATABASES; | payments_db| | payroll_db| +------------+ + -- Lists databases with name starting with string pattern `pay` SHOW DATABASES LIKE 'pay*'; +------------+ @@ -63,6 +68,7 @@ SHOW DATABASES LIKE 'pay*'; | payments_db| | payroll_db| +------------+ + -- Lists all databases. Keywords SCHEMAS and DATABASES are interchangeable. SHOW SCHEMAS; +------------+ @@ -73,7 +79,9 @@ SHOW SCHEMAS; | payroll_db| +------------+ {% endhighlight %} + ### Related Statements -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) -- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) -- [ALTER DATABASE](sql-ref-syntax-ddl-alter-database.html) + + * [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) + * [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) + * [ALTER DATABASE](sql-ref-syntax-ddl-alter-database.html) diff --git a/docs/sql-ref-syntax-aux-show-functions.md b/docs/sql-ref-syntax-aux-show-functions.md index d6f9df9896afe..da33d999f0b38 100644 --- a/docs/sql-ref-syntax-aux-show-functions.md +++ b/docs/sql-ref-syntax-aux-show-functions.md @@ -20,6 +20,7 @@ license: | --- ### Description + Returns the list of functions after applying an optional regex pattern. Given number of functions supported by Spark is quite large, this statement in conjunction with [describe function](sql-ref-syntax-aux-describe-function.html) @@ -27,11 +28,13 @@ may be used to quickly find the function and understand its usage. The `LIKE` clause is optional and supported only for compatibility with other systems. ### Syntax + {% highlight sql %} SHOW [ function_kind ] FUNCTIONS ( [ LIKE ] function_name | regex_pattern ) {% endhighlight %} ### Parameters +
    function_kind
    @@ -66,6 +69,7 @@ SHOW [ function_kind ] FUNCTIONS ( [ LIKE ] function_name | regex_pattern )
    ### Examples + {% highlight sql %} -- List a system function `trim` by searching both user defined and system -- defined functions. @@ -73,7 +77,7 @@ SHOW FUNCTIONS trim; +--------+ |function| +--------+ - |trim | + | trim| +--------+ -- List a system function `concat` by searching system defined functions. @@ -81,7 +85,7 @@ SHOW SYSTEM FUNCTIONS concat; +--------+ |function| +--------+ - |concat | + | concat| +--------+ -- List a qualified function `max` from database `salesdb`. @@ -89,30 +93,31 @@ SHOW SYSTEM FUNCTIONS salesdb.max; +--------+ |function| +--------+ - |max | + | max| +--------+ -- List all functions starting with `t` SHOW FUNCTIONS LIKE 't*'; +-----------------+ - |function | + | function| +-----------------+ - |tan | - |tanh | - |timestamp | - |tinyint | - |to_csv | - |to_date | - |to_json | - |to_timestamp | + | tan| + | tanh| + | timestamp| + | tinyint| + | to_csv| + | to_date| + | to_json| + | to_timestamp| |to_unix_timestamp| - |to_utc_timestamp | - |transform | - |transform_keys | - |transform_values | - |translate | - |trim | - |trunc | + | to_utc_timestamp| + | transform| + | transform_keys| + | transform_values| + | translate| + | trim| + | trunc| + | typeof| +-----------------+ -- List all functions starting with `yea` or `windo` @@ -120,8 +125,8 @@ SHOW FUNCTIONS LIKE 'yea*|windo*'; +--------+ |function| +--------+ - |window | - |year | + | window| + | year| +--------+ -- Use normal regex pattern to list function names that has 4 characters @@ -130,10 +135,11 @@ SHOW FUNCTIONS LIKE 't[a-z][a-z][a-z]'; +--------+ |function| +--------+ - |tanh | - |trim | + | tanh| + | trim| +--------+ {% endhighlight %} -### Related statements -- [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) +### Related Statements + + * [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) diff --git a/docs/sql-ref-syntax-aux-show-partitions.md b/docs/sql-ref-syntax-aux-show-partitions.md index 6c8401e8e4718..31b881ea8f141 100644 --- a/docs/sql-ref-syntax-aux-show-partitions.md +++ b/docs/sql-ref-syntax-aux-show-partitions.md @@ -18,6 +18,7 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description The `SHOW PARTITIONS` statement is used to list partitions of a table. An optional @@ -25,11 +26,13 @@ partition spec may be specified to return the partitions matching the supplied partition spec. ### Syntax + {% highlight sql %} SHOW PARTITIONS table_identifier [ partition_spec ] {% endhighlight %} ### Parameters +
    table_identifier
    @@ -53,6 +56,7 @@ SHOW PARTITIONS table_identifier [ partition_spec ]
    ### Examples + {% highlight sql %} -- create a partitioned table and insert a few rows. USE salesdb; @@ -64,27 +68,27 @@ INSERT INTO customer PARTITION (state = 'AZ', city = 'Peoria') VALUES (300, 'Dan -- Lists all partitions for table `customer` SHOW PARTITIONS customer; +----------------------+ - |partition | + | partition| +----------------------+ - |state=AZ/city=Peoria | - |state=CA/city=Fremont | + | state=AZ/city=Peoria| + | state=CA/city=Fremont| |state=CA/city=San Jose| +----------------------+ -- Lists all partitions for the qualified table `customer` SHOW PARTITIONS salesdb.customer; +----------------------+ - |partition | + | partition| +----------------------+ - |state=AZ/city=Peoria | - |state=CA/city=Fremont | + | state=AZ/city=Peoria| + | state=CA/city=Fremont| |state=CA/city=San Jose| +----------------------+ -- Specify a full partition spec to list specific partition SHOW PARTITIONS customer PARTITION (state = 'CA', city = 'Fremont'); +---------------------+ - |partition | + | partition| +---------------------+ |state=CA/city=Fremont| +---------------------+ @@ -92,23 +96,24 @@ SHOW PARTITIONS customer PARTITION (state = 'CA', city = 'Fremont'); -- Specify a partial partition spec to list the specific partitions SHOW PARTITIONS customer PARTITION (state = 'CA'); +----------------------+ - |partition | + | partition| +----------------------+ - |state=CA/city=Fremont | + | state=CA/city=Fremont| |state=CA/city=San Jose| +----------------------+ -- Specify a partial spec to list specific partition SHOW PARTITIONS customer PARTITION (city = 'San Jose'); +----------------------+ - |partition | + | partition| +----------------------+ |state=CA/city=San Jose| +----------------------+ {% endhighlight %} -### Related statements -- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) -- [INSERT STATEMENT](sql-ref-syntax-dml-insert.html) -- [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) -- [SHOW TABLE](sql-ref-syntax-aux-show-table.html) +### Related Statements + + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) + * [INSERT STATEMENT](sql-ref-syntax-dml-insert.html) + * [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) + * [SHOW TABLE](sql-ref-syntax-aux-show-table.html) diff --git a/docs/sql-ref-syntax-aux-show-table.md b/docs/sql-ref-syntax-aux-show-table.md index 49696585ba581..1aa44d3ab30ea 100644 --- a/docs/sql-ref-syntax-aux-show-table.md +++ b/docs/sql-ref-syntax-aux-show-table.md @@ -18,6 +18,7 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description `SHOW TABLE EXTENDED` will show information for all tables matching the given regular expression. @@ -30,12 +31,14 @@ information such as `Partition Parameters` and `Partition Statistics`. Note that cannot be used with a partition specification. ### Syntax + {% highlight sql %} SHOW TABLE EXTENDED [ IN | FROM database_name ] LIKE 'identifier_with_wildcards' [ partition_spec ] {% endhighlight %} ### Parameters +
    IN|FROM database_name
    @@ -61,123 +64,123 @@ SHOW TABLE EXTENDED [ IN | FROM database_name ] LIKE 'identifier_with_wildcards'
    + ### Examples + {% highlight sql %} -- Assumes `employee` table created with partitioned by column `grade` --- +-------+--------+--+ --- | name | grade | --- +-------+--------+--+ --- | sam | 1 | --- | suj | 2 | --- +-------+--------+--+ +CREATE TABLE employee(name STRING, grade INT) PARTITIONED BY (grade); +INSERT INTO employee PARTITION (grade = 1) VALUES ('sam'); +INSERT INTO employee PARTITION (grade = 2) VALUES ('suj'); -- Show the details of the table -SHOW TABLE EXTENDED LIKE `employee`; -+--------+---------+-----------+--------------------------------------------------------------- -|database|tableName|isTemporary| information -+--------+---------+-----------+--------------------------------------------------------------- -|default |employee |false |Database: default - Table: employee - Owner: root - Created Time: Fri Aug 30 15:10:21 IST 2019 - Last Access: Thu Jan 01 05:30:00 IST 1970 - Created By: Spark 3.0.0-SNAPSHOT - Type: MANAGED - Provider: hive - Table Properties: [transient_lastDdlTime=1567158021] - Location: file:/opt/spark1/spark/spark-warehouse/employee - Serde Library: org.apache.hadoop.hive.serde2.lazy - .LazySimpleSerDe - InputFormat: org.apache.hadoop.mapred.TextInputFormat - OutputFormat: org.apache.hadoop.hive.ql.io - .HiveIgnoreKeyTextOutputFormat - Storage Properties: [serialization.format=1] - Partition Provider: Catalog - Partition Columns: [`grade`] - Schema: root - |-- name: string (nullable = true) - |-- grade: integer (nullable = true) - -+--------+---------+-----------+--------------------------------------------------------------- +SHOW TABLE EXTENDED LIKE 'employee'; + +--------+---------+-----------+--------------------------------------------------------------+ + |database|tableName|isTemporary| information | + +--------+---------+-----------+--------------------------------------------------------------+ + |default |employee |false |Database: default + Table: employee + Owner: root + Created Time: Fri Aug 30 15:10:21 IST 2019 + Last Access: Thu Jan 01 05:30:00 IST 1970 + Created By: Spark 3.0.0-SNAPSHOT + Type: MANAGED + Provider: hive + Table Properties: [transient_lastDdlTime=1567158021] + Location: file:/opt/spark1/spark/spark-warehouse/employee + Serde Library: org.apache.hadoop.hive.serde2.lazy + .LazySimpleSerDe + InputFormat: org.apache.hadoop.mapred.TextInputFormat + OutputFormat: org.apache.hadoop.hive.ql.io + .HiveIgnoreKeyTextOutputFormat + Storage Properties: [serialization.format=1] + Partition Provider: Catalog + Partition Columns: [`grade`] + Schema: root + |-- name: string (nullable = true) + |-- grade: integer (nullable = true) + + +--------+---------+-----------+--------------------------------------------------------------+ -- showing the multiple table details with pattern matching SHOW TABLE EXTENDED LIKE `employe*`; -+--------+---------+-----------+--------------------------------------------------------------- -|database|tableName|isTemporary| information -+--------+---------+-----------+--------------------------------------------------------------- -|default |employee |false |Database: default - Table: employee - Owner: root - Created Time: Fri Aug 30 15:10:21 IST 2019 - Last Access: Thu Jan 01 05:30:00 IST 1970 - Created By: Spark 3.0.0-SNAPSHOT - Type: MANAGED - Provider: hive - Table Properties: [transient_lastDdlTime=1567158021] - Location: file:/opt/spark1/spark/spark-warehouse/employee - Serde Library: org.apache.hadoop.hive.serde2.lazy - .LazySimpleSerDe - InputFormat: org.apache.hadoop.mapred.TextInputFormat - OutputFormat: org.apache.hadoop.hive.ql.io - .HiveIgnoreKeyTextOutputFormat - Storage Properties: [serialization.format=1] - Partition Provider: Catalog - Partition Columns: [`grade`] - Schema: root - |-- name: string (nullable = true) - |-- grade: integer (nullable = true) - -|default |employee1|false |Database: default - Table: employee1 - Owner: root - Created Time: Fri Aug 30 15:22:33 IST 2019 - Last Access: Thu Jan 01 05:30:00 IST 1970 - Created By: Spark 3.0.0-SNAPSHOT - Type: MANAGED - Provider: hive - Table Properties: [transient_lastDdlTime=1567158753] - Location: file:/opt/spark1/spark/spark-warehouse/employee1 - Serde Library: org.apache.hadoop.hive.serde2.lazy - .LazySimpleSerDe - InputFormat: org.apache.hadoop.mapred.TextInputFormat - OutputFormat: org.apache.hadoop.hive.ql.io - .HiveIgnoreKeyTextOutputFormat - Storage Properties: [serialization.format=1] - Partition Provider: Catalog - Schema: root - |-- name: string (nullable = true) - -+--------+---------+----------+---------------------------------------------------------------- + +--------+---------+-----------+--------------------------------------------------------------+ + |database|tableName|isTemporary| information | + +--------+---------+-----------+--------------------------------------------------------------+ + |default |employee |false |Database: default + Table: employee + Owner: root + Created Time: Fri Aug 30 15:10:21 IST 2019 + Last Access: Thu Jan 01 05:30:00 IST 1970 + Created By: Spark 3.0.0-SNAPSHOT + Type: MANAGED + Provider: hive + Table Properties: [transient_lastDdlTime=1567158021] + Location: file:/opt/spark1/spark/spark-warehouse/employee + Serde Library: org.apache.hadoop.hive.serde2.lazy + .LazySimpleSerDe + InputFormat: org.apache.hadoop.mapred.TextInputFormat + OutputFormat: org.apache.hadoop.hive.ql.io + .HiveIgnoreKeyTextOutputFormat + Storage Properties: [serialization.format=1] + Partition Provider: Catalog + Partition Columns: [`grade`] + Schema: root + |-- name: string (nullable = true) + |-- grade: integer (nullable = true) + + |default |employee1|false |Database: default + Table: employee1 + Owner: root + Created Time: Fri Aug 30 15:22:33 IST 2019 + Last Access: Thu Jan 01 05:30:00 IST 1970 + Created By: Spark 3.0.0-SNAPSHOT + Type: MANAGED + Provider: hive + Table Properties: [transient_lastDdlTime=1567158753] + Location: file:/opt/spark1/spark/spark-warehouse/employee1 + Serde Library: org.apache.hadoop.hive.serde2.lazy + .LazySimpleSerDe + InputFormat: org.apache.hadoop.mapred.TextInputFormat + OutputFormat: org.apache.hadoop.hive.ql.io + .HiveIgnoreKeyTextOutputFormat + Storage Properties: [serialization.format=1] + Partition Provider: Catalog + Schema: root + |-- name: string (nullable = true) + + +--------+---------+----------+---------------------------------------------------------------+ -- show partition file system details SHOW TABLE EXTENDED IN `default` LIKE `employee` PARTITION (`grade=1`); -+--------+---------+-----------+--------------------------------------------------------------- -|database|tableName|isTemporary| information -+--------+---------+-----------+--------------------------------------------------------------- -|default |employee |false | Partition Values: [grade=1] - Location: file:/opt/spark1/spark/spark-warehouse/employee - /grade=1 - Serde Library: org.apache.hadoop.hive.serde2.lazy - .LazySimpleSerDe - InputFormat: org.apache.hadoop.mapred.TextInputFormat - OutputFormat: org.apache.hadoop.hive.ql.io - .HiveIgnoreKeyTextOutputFormat - Storage Properties: [serialization.format=1] - Partition Parameters: {rawDataSize=-1, numFiles=1, - transient_lastDdlTime=1567158221, totalSize=4, - COLUMN_STATS_ACCURATE=false, numRows=-1} - Created Time: Fri Aug 30 15:13:41 IST 2019 - Last Access: Thu Jan 01 05:30:00 IST 1970 - Partition Statistics: 4 bytes - | -+--------+---------+-----------+--------------------------------------------------------------- + +--------+---------+-----------+--------------------------------------------------------------+ + |database|tableName|isTemporary| information | + +--------+---------+-----------+--------------------------------------------------------------+ + |default |employee |false |Partition Values: [grade=1] + Location: file:/opt/spark1/spark/spark-warehouse/employee + /grade=1 + Serde Library: org.apache.hadoop.hive.serde2.lazy + .LazySimpleSerDe + InputFormat: org.apache.hadoop.mapred.TextInputFormat + OutputFormat: org.apache.hadoop.hive.ql.io + .HiveIgnoreKeyTextOutputFormat + Storage Properties: [serialization.format=1] + Partition Parameters: {rawDataSize=-1, numFiles=1, + transient_lastDdlTime=1567158221, totalSize=4, + COLUMN_STATS_ACCURATE=false, numRows=-1} + Created Time: Fri Aug 30 15:13:41 IST 2019 + Last Access: Thu Jan 01 05:30:00 IST 1970 + Partition Statistics: 4 bytes + | + +--------+---------+-----------+--------------------------------------------------------------+ -- show partition file system details with regex fails as shown below SHOW TABLE EXTENDED IN `default` LIKE `empl*` PARTITION (`grade=1`); -Error: Error running query: org.apache.spark.sql.catalyst.analysis.NoSuchTableException: - Table or view 'emplo*' not found in database 'default'; (state=,code=0) - + Error: Error running query: org.apache.spark.sql.catalyst.analysis.NoSuchTableException: + Table or view 'emplo*' not found in database 'default'; (state=,code=0) {% endhighlight %} + ### Related Statements -- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) -- [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) + + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) + * [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) diff --git a/docs/sql-ref-syntax-aux-show-tables.md b/docs/sql-ref-syntax-aux-show-tables.md index 2a078abf911e7..0b7062ec8eff7 100644 --- a/docs/sql-ref-syntax-aux-show-tables.md +++ b/docs/sql-ref-syntax-aux-show-tables.md @@ -18,6 +18,7 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description The `SHOW TABLES` statement returns all the tables for an optionally specified database. @@ -26,11 +27,13 @@ pattern. If no database is specified then the tables are returned from the current database. ### Syntax + {% highlight sql %} SHOW TABLES [ { FROM | IN } database_name ] [ LIKE 'regex_pattern' ] {% endhighlight %} ### Parameters +
    { FROM | IN } database_name
    @@ -50,58 +53,59 @@ SHOW TABLES [ { FROM | IN } database_name ] [ LIKE 'regex_pattern' ]
    ### Example + {% highlight sql %} -- List all tables in default database SHOW TABLES; - +-----------+------------+--------------+--+ - | database | tableName | isTemporary | - +-----------+------------+--------------+--+ - | default | sam | false | - | default | sam1 | false | - | default | suj | false | - +-----------+------------+--------------+--+ + +--------+---------+-----------+ + |database|tableName|isTemporary| + +--------+---------+-----------+ + | default| sam| false| + | default| sam1| false| + | default| suj| false| + +--------+---------+-----------+ -- List all tables from userdb database SHOW TABLES FROM userdb; - +-----------+------------+--------------+--+ - | database | tableName | isTemporary | - +-----------+------------+--------------+--+ - | userdb | user1 | false | - | userdb | user2 | false | - +-----------+------------+--------------+--+ + +--------+---------+-----------+ + |database|tableName|isTemporary| + +--------+---------+-----------+ + | userdb| user1| false| + | userdb| user2| false| + +--------+---------+-----------+ -- List all tables in userdb database SHOW TABLES IN userdb; - +-----------+------------+--------------+--+ - | database | tableName | isTemporary | - +-----------+------------+--------------+--+ - | userdb | user1 | false | - | userdb | user2 | false | - +-----------+------------+--------------+--+ + +--------+---------+-----------+ + |database|tableName|isTemporary| + +--------+---------+-----------+ + | userdb| user1| false| + | userdb| user2| false| + +--------+---------+-----------+ -- List all tables from default database matching the pattern `sam*` SHOW TABLES FROM default LIKE 'sam*'; - +-----------+------------+--------------+--+ - | database | tableName | isTemporary | - +-----------+------------+--------------+--+ - | default | sam | false | - | default | sam1 | false | - +-----------+------------+--------------+--+ + +--------+---------+-----------+ + |database|tableName|isTemporary| + +--------+---------+-----------+ + | default| sam| false| + | default| sam1| false| + +--------+---------+-----------+ -- List all tables matching the pattern `sam*|suj` -SHOW TABLES LIKE 'sam*|suj'; - +-----------+------------+--------------+--+ - | database | tableName | isTemporary | - +-----------+------------+--------------+--+ - | default | sam | false | - | default | sam1 | false | - | default | suj | false | - +-----------+------------+--------------+--+ - +SHOW TABLES LIKE 'sam*|suj'; + +--------+---------+-----------+ + |database|tableName|isTemporary| + +--------+---------+-----------+ + | default| sam| false| + | default| sam1| false| + | default| suj| false| + +--------+---------+-----------+ {% endhighlight %} -### Related statements -- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) -- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) -- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) -- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) +### Related Statements + + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) + * [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) + * [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) + * [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) diff --git a/docs/sql-ref-syntax-aux-show-tblproperties.md b/docs/sql-ref-syntax-aux-show-tblproperties.md index 451fd4abc85bb..26e57ef0aba24 100644 --- a/docs/sql-ref-syntax-aux-show-tblproperties.md +++ b/docs/sql-ref-syntax-aux-show-tblproperties.md @@ -20,16 +20,19 @@ license: | --- ### Description + This statement returns the value of a table property given an optional value for a property key. If no key is specified then all the properties are returned. ### Syntax + {% highlight sql %} SHOW TBLPROPERTIES table_identifier [ ( unquoted_property_key | property_key_as_string_literal ) ] {% endhighlight %} ### Parameters +
    table_identifier
    @@ -64,19 +67,20 @@ SHOW TBLPROPERTIES table_identifier properties are: `numFiles`, `numPartitions`, `numRows`. ### Examples + {% highlight sql %} -- create a table `customer` in database `salesdb` USE salesdb; CREATE TABLE customer(cust_code INT, name VARCHAR(100), cust_addr STRING) - TBLPROPERTIES ('created.by.user' = 'John', 'created.date' = '01-01-2001'); + TBLPROPERTIES ('created.by.user' = 'John', 'created.date' = '01-01-2001'); -- show all the user specified properties for table `customer` SHOW TBLPROPERTIES customer; +---------------------+----------+ - |key |value | + | key| value| +---------------------+----------+ - |created.by.user |John | - |created.date |01-01-2001| + | created.by.user| John| + | created.date|01-01-2001| |transient_lastDdlTime|1567554931| +---------------------+----------+ @@ -84,10 +88,10 @@ SHOW TBLPROPERTIES customer; -- in database `salesdb` SHOW TBLPROPERTIES salesdb.customer; +---------------------+----------+ - |key |value | + | key| value| +---------------------+----------+ - |created.by.user |John | - |created.date |01-01-2001| + | created.by.user| John| + | created.date|01-01-2001| |transient_lastDdlTime|1567554931| +---------------------+----------+ @@ -96,20 +100,21 @@ SHOW TBLPROPERTIES customer (created.by.user); +-----+ |value| +-----+ - |John | + | John| +-----+ -- show value for property `created.date`` specified as string literal SHOW TBLPROPERTIES customer ('created.date'); +----------+ - |value | + | value| +----------+ |01-01-2001| +----------+ {% endhighlight %} ### Related Statements -- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) -- [ALTER TABLE SET TBLPROPERTIES](sql-ref-syntax-ddl-alter-table.html) -- [SHOW TABLES](sql-ref-syntax-aux-show-tables.html) -- [SHOW TABLE EXTENDED](sql-ref-syntax-aux-show-table.html) + + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) + * [ALTER TABLE SET TBLPROPERTIES](sql-ref-syntax-ddl-alter-table.html) + * [SHOW TABLES](sql-ref-syntax-aux-show-tables.html) + * [SHOW TABLE EXTENDED](sql-ref-syntax-aux-show-table.html) diff --git a/docs/sql-ref-syntax-aux-show-views.md b/docs/sql-ref-syntax-aux-show-views.md new file mode 100644 index 0000000000000..aec3716c2889f --- /dev/null +++ b/docs/sql-ref-syntax-aux-show-views.md @@ -0,0 +1,122 @@ +--- +layout: global +title: SHOW VIEWS +displayTitle: SHOW VIEWS +license: | + 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. +--- + +### Description + +The `SHOW VIEWS` statement returns all the views for an optionally specified database. +Additionally, the output of this statement may be filtered by an optional matching +pattern. If no database is specified then the views are returned from the +current database. If the specified database is global temporary view database, we will +list global temporary views. Note that the command also lists local temporary views +regardless of a given database. + +### Syntax +{% highlight sql %} +SHOW VIEWS [ { FROM | IN } database_name ] [ LIKE 'regex_pattern' ] +{% endhighlight %} + +### Parameters +
    +
    { FROM | IN } database_name
    +
    + Specifies the database name from which views are listed. +
    +
    LIKE regex_pattern
    +
    + Specifies the regular expression pattern that is used to filter out unwanted views. +
      +
    • Except for `*` and `|` character, the pattern works like a regex.
    • +
    • `*` alone matches 0 or more characters and `|` is used to separate multiple different regexes, + any of which can match.
    • +
    • The leading and trailing blanks are trimmed in the input pattern before processing.
    • +
    +
    +
    + +### Example +{% highlight sql %} +-- Create views in different databases, also create global/local temp views. +CREATE VIEW sam AS SELECT id, salary FROM employee WHERE name = 'sam'; +CREATE VIEW sam1 AS SELECT id, salary FROM employee WHERE name = 'sam1'; +CREATE VIEW suj AS SELECT id, salary FROM employee WHERE name = 'suj'; +USE userdb; +CREATE VIEW user1 AS SELECT id, salary FROM default.employee WHERE name = 'user1'; +CREATE VIEW user2 AS SELECT id, salary FROM default.employee WHERE name = 'user2'; +USE default; +CREATE GLOBAL TEMP VIEW temp1 AS SELECT 1 as col1; +CREATE TEMP VIEW temp2 AS SELECT 1 as col1; + +-- List all views in default database +SHOW VIEWS; + +-------------+------------+--------------+--+ + | namespace | viewName | isTemporary | + +-------------+------------+--------------+--+ + | default | sam | false | + | default | sam1 | false | + | default | suj | false | + | | temp2 | true | + +-------------+------------+--------------+--+ + +-- List all views from userdb database +SHOW VIEWS FROM userdb; + +-------------+------------+--------------+--+ + | namespace | viewName | isTemporary | + +-------------+------------+--------------+--+ + | userdb | user1 | false | + | userdb | user2 | false | + | | temp2 | true | + +-------------+------------+--------------+--+ + +-- List all views in global temp view database +SHOW VIEWS IN global_temp; + +-------------+------------+--------------+--+ + | namespace | viewName | isTemporary | + +-------------+------------+--------------+--+ + | global_temp | temp1 | true | + | | temp2 | true | + +-------------+------------+--------------+--+ + +-- List all views from default database matching the pattern `sam*` +SHOW VIEWS FROM default LIKE 'sam*'; + +-----------+------------+--------------+--+ + | namespace | viewName | isTemporary | + +-----------+------------+--------------+--+ + | default | sam | false | + | default | sam1 | false | + +-----------+------------+--------------+--+ + +-- List all views from the current database matching the pattern `sam|suj|temp*` +SHOW VIEWS LIKE 'sam|suj|temp*'; + +-------------+------------+--------------+--+ + | namespace | viewName | isTemporary | + +-------------+------------+--------------+--+ + | default | sam | false | + | default | suj | false | + | | temp2 | true | + +-------------+------------+--------------+--+ + +{% endhighlight %} + +### Related statements +- [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) +- [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) +- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) +- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) diff --git a/docs/sql-ref-syntax-aux-show.md b/docs/sql-ref-syntax-aux-show.md index f6d700acf9eef..424fe71370897 100644 --- a/docs/sql-ref-syntax-aux-show.md +++ b/docs/sql-ref-syntax-aux-show.md @@ -18,11 +18,13 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- -* [SHOW COLUMNS](sql-ref-syntax-aux-show-columns.html) -* [SHOW DATABASES](sql-ref-syntax-aux-show-databases.html) -* [SHOW FUNCTIONS](sql-ref-syntax-aux-show-functions.html) -* [SHOW TABLE EXTENDED](sql-ref-syntax-aux-show-table.html) -* [SHOW TABLES](sql-ref-syntax-aux-show-tables.html) -* [SHOW TBLPROPERTIES](sql-ref-syntax-aux-show-tblproperties.html) -* [SHOW PARTITIONS](sql-ref-syntax-aux-show-partitions.html) -* [SHOW CREATE TABLE](sql-ref-syntax-aux-show-create-table.html) + + * [SHOW COLUMNS](sql-ref-syntax-aux-show-columns.html) + * [SHOW DATABASES](sql-ref-syntax-aux-show-databases.html) + * [SHOW FUNCTIONS](sql-ref-syntax-aux-show-functions.html) + * [SHOW TABLE EXTENDED](sql-ref-syntax-aux-show-table.html) + * [SHOW TABLES](sql-ref-syntax-aux-show-tables.html) + * [SHOW TBLPROPERTIES](sql-ref-syntax-aux-show-tblproperties.html) + * [SHOW PARTITIONS](sql-ref-syntax-aux-show-partitions.html) + * [SHOW CREATE TABLE](sql-ref-syntax-aux-show-create-table.html) + * [SHOW VIEWS](sql-ref-syntax-aux-show-views.html) diff --git a/docs/sql-ref-syntax-aux.md b/docs/sql-ref-syntax-aux.md index ba09d70b437a9..3cd758f076e99 100644 --- a/docs/sql-ref-syntax-aux.md +++ b/docs/sql-ref-syntax-aux.md @@ -21,9 +21,9 @@ license: | Besides the major SQL statements such as Data Definition Statements, Data Manipulation Statements and Data Retrieval Statements, Spark SQL also supports the following Auxiliary Statements: -- [ANALYZE](sql-ref-syntax-aux-analyze.html) -- [CACHE](sql-ref-syntax-aux-cache.html) -- [DESCRIBE](sql-ref-syntax-aux-describe.html) -- [SHOW](sql-ref-syntax-aux-show.html) -- [CONFIGURATION MANAGEMENT](sql-ref-syntax-aux-conf-mgmt.html) -- [RESOURCE MANAGEMENT](sql-ref-syntax-aux-resource-mgmt.html) + * [ANALYZE](sql-ref-syntax-aux-analyze.html) + * [CACHE](sql-ref-syntax-aux-cache.html) + * [DESCRIBE](sql-ref-syntax-aux-describe.html) + * [SHOW](sql-ref-syntax-aux-show.html) + * [CONFIGURATION MANAGEMENT](sql-ref-syntax-aux-conf-mgmt.html) + * [RESOURCE MANAGEMENT](sql-ref-syntax-aux-resource-mgmt.html) diff --git a/docs/sql-ref-syntax-ddl-alter-database.md b/docs/sql-ref-syntax-ddl-alter-database.md index a32343674feb0..520aba35567e8 100644 --- a/docs/sql-ref-syntax-ddl-alter-database.md +++ b/docs/sql-ref-syntax-ddl-alter-database.md @@ -18,7 +18,9 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description + You can alter metadata associated with a database by setting `DBPROPERTIES`. The specified property values override any existing value with the same property name. Please note that the usage of `SCHEMA` and `DATABASE` are interchangeable and one can be used in place of the other. An error message @@ -26,18 +28,21 @@ is issued if the database is not found in the system. This command is mostly use for a database and may be used for auditing purposes. ### Syntax + {% highlight sql %} ALTER { DATABASE | SCHEMA } database_name SET DBPROPERTIES ( property_name = property_value, ... ) {% endhighlight %} ### Parameters +
    database_name
    Specifies the name of the database to be altered.
    ### Examples + {% highlight sql %} -- Creates a database named `inventory`. CREATE DATABASE inventory; @@ -47,16 +52,16 @@ ALTER DATABASE inventory SET DBPROPERTIES ('Edited-by' = 'John', 'Edit-date' = ' -- Verify that properties are set. DESCRIBE DATABASE EXTENDED inventory; - - +-------------------------+--------------------------------------------+ - |database_description_item|database_description_value | - +-------------------------+--------------------------------------------+ - |Database Name |inventory | - |Description | | - |Location |file:/temp/spark-warehouse/inventory.db | - |Properties |((Edit-date,01/01/2001), (Edited-by,John)) | - +-------------------------+--------------------------------------------+ + +-------------------------+------------------------------------------+ + |database_description_item| database_description_value| + +-------------------------+------------------------------------------+ + | Database Name| inventory| + | Description| | + | Location| file:/temp/spark-warehouse/inventory.db| + | Properties|((Edit-date,01/01/2001), (Edited-by,John))| + +-------------------------+------------------------------------------+ {% endhighlight %} ### Related Statements -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) + + * [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md index 2dd808b131ef2..edb081b7f45c0 100644 --- a/docs/sql-ref-syntax-ddl-alter-table.md +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -20,12 +20,15 @@ license: | --- ### Description + `ALTER TABLE` statement changes the schema or properties of a table. ### RENAME + `ALTER TABLE RENAME TO` statement changes the table name of an existing table in the database. #### Syntax + {% highlight sql %} ALTER TABLE table_identifier RENAME TO table_identifier @@ -33,6 +36,7 @@ ALTER TABLE table_identifier partition_spec RENAME TO partition_spec {% endhighlight %} #### Parameters +
    table_identifier
    @@ -55,16 +59,18 @@ ALTER TABLE table_identifier partition_spec RENAME TO partition_spec
    - ### ADD COLUMNS + `ALTER TABLE ADD COLUMNS` statement adds mentioned columns to an existing table. #### Syntax + {% highlight sql %} ALTER TABLE table_identifier ADD COLUMNS ( col_spec [ , col_spec ... ] ) {% endhighlight %} #### Parameters +
    table_identifier
    @@ -81,16 +87,18 @@ ALTER TABLE table_identifier ADD COLUMNS ( col_spec [ , col_spec ... ] )
    Specifies the columns to be added to be renamed.
    - ### ALTER OR CHANGE COLUMN + `ALTER TABLE ALTER COLUMN` or `ALTER TABLE CHANGE COLUMN` statement changes column's comment. #### Syntax + {% highlight sql %} ALTER TABLE table_identifier { ALTER | CHANGE } [ COLUMN ] col_spec alterColumnAction {% endhighlight %} #### Parameters +
    table_identifier
    @@ -118,19 +126,21 @@ ALTER TABLE table_identifier { ALTER | CHANGE } [ COLUMN ] col_spec alterColumnA
    - ### ADD AND DROP PARTITION #### ADD PARTITION + `ALTER TABLE ADD` statement adds partition to the partitioned table. ##### Syntax + {% highlight sql %} ALTER TABLE table_identifier ADD [IF NOT EXISTS] ( partition_spec [ partition_spec ... ] ) {% endhighlight %} ##### Parameters +
    table_identifier
    @@ -154,14 +164,17 @@ ALTER TABLE table_identifier ADD [IF NOT EXISTS]
    #### DROP PARTITION + `ALTER TABLE DROP` statement drops the partition of the table. ##### Syntax + {% highlight sql %} ALTER TABLE table_identifier DROP [ IF EXISTS ] partition_spec [PURGE] {% endhighlight %} ##### Parameters +
    table_identifier
    @@ -183,35 +196,35 @@ ALTER TABLE table_identifier DROP [ IF EXISTS ] partition_spec [PURGE]
    - ### SET AND UNSET #### SET TABLE PROPERTIES + `ALTER TABLE SET` command is used for setting the table properties. If a particular property was already set, this overrides the old value with the new one. `ALTER TABLE UNSET` is used to drop the table property. ##### Syntax -{% highlight sql %} ---Set Table Properties +{% highlight sql %} +-- Set Table Properties ALTER TABLE table_identifier SET TBLPROPERTIES ( key1 = val1, key2 = val2, ... ) ---Unset Table Properties +-- Unset Table Properties ALTER TABLE table_identifier UNSET TBLPROPERTIES [ IF EXISTS ] ( key1, key2, ... ) - {% endhighlight %} #### SET SERDE + `ALTER TABLE SET` command is used for setting the SERDE or SERDE properties in Hive tables. If a particular property was already set, this overrides the old value with the new one. ##### Syntax -{% highlight sql %} ---Set SERDE Properties +{% highlight sql %} +-- Set SERDE Properties ALTER TABLE table_identifier [ partition_spec ] SET SERDEPROPERTIES ( key1 = val1, key2 = val2, ... ) @@ -221,21 +234,22 @@ ALTER TABLE table_identifier [ partition_spec ] SET SERDE serde_class_name {% endhighlight %} #### SET LOCATION And SET FILE FORMAT + `ALTER TABLE SET` command can also be used for changing the file location and file format for existing tables. ##### Syntax -{% highlight sql %} ---Changing File Format +{% highlight sql %} +-- Changing File Format ALTER TABLE table_identifier [ partition_spec ] SET FILEFORMAT file_format ---Changing File Location +-- Changing File Location ALTER TABLE table_identifier [ partition_spec ] SET LOCATION 'new_location' - {% endhighlight %} #### Parameters +
    table_identifier
    @@ -263,205 +277,198 @@ ALTER TABLE table_identifier [ partition_spec ] SET LOCATION 'new_location'
    Specifies the SERDE properties to be set.
    - ### Examples -{% highlight sql %} ---RENAME table +{% highlight sql %} +-- RENAME table DESC student; -+--------------------------+------------+----------+--+ -| col_name | data_type | comment | -+--------------------------+------------+----------+--+ -| name | string | NULL | -| rollno | int | NULL | -| age | int | NULL | -| # Partition Information | | | -| # col_name | data_type | comment | -| age | int | NULL | -+--------------------------+------------+----------+--+ + +-----------------------+---------+-------+ + | col_name|data_type|comment| + +-----------------------+---------+-------+ + | name| string| NULL| + | rollno| int| NULL| + | age| int| NULL| + |# Partition Information| | | + | # col_name|data_type|comment| + | age| int| NULL| + +-----------------------+---------+-------+ ALTER TABLE Student RENAME TO StudentInfo; ---After Renaming the table - +-- After Renaming the table DESC StudentInfo; -+--------------------------+------------+----------+--+ -| col_name | data_type | comment | -+--------------------------+------------+----------+--+ -| name | string | NULL | -| rollno | int | NULL | -| age | int | NULL | -| # Partition Information | | | -| # col_name | data_type | comment | -| age | int | NULL | -+--------------------------+------------+----------+--+ - ---RENAME partition + +-----------------------+---------+-------+ + | col_name|data_type|comment| + +-----------------------+---------+-------+ + | name| string| NULL| + | rollno| int| NULL| + | age| int| NULL| + |# Partition Information| | | + | # col_name|data_type|comment| + | age| int| NULL| + +-----------------------+---------+-------+ + +-- RENAME partition SHOW PARTITIONS StudentInfo; -+------------+--+ -| partition | -+------------+--+ -| age=10 | -| age=11 | -| age=12 | -+------------+--+ + +---------+ + |partition| + +---------+ + | age=10| + | age=11| + | age=12| + +---------+ ALTER TABLE default.StudentInfo PARTITION (age='10') RENAME TO PARTITION (age='15'); ---After renaming Partition +-- After renaming Partition SHOW PARTITIONS StudentInfo; -+------------+--+ -| partition | -+------------+--+ -| age=11 | -| age=12 | -| age=15 | -+------------+--+ + +---------+ + |partition| + +---------+ + | age=11| + | age=12| + | age=15| + +---------+ -- Add new columns to a table - DESC StudentInfo; -+--------------------------+------------+----------+--+ -| col_name | data_type | comment | -+--------------------------+------------+----------+--+ -| name | string | NULL | -| rollno | int | NULL | -| age | int | NULL | -| # Partition Information | | | -| # col_name | data_type | comment | -| age | int | NULL | -+--------------------------+------------+----------+ + +-----------------------+---------+-------+ + | col_name|data_type|comment| + +-----------------------+---------+-------+ + | name| string| NULL| + | rollno| int| NULL| + | age| int| NULL| + |# Partition Information| | | + | # col_name|data_type|comment| + | age| int| NULL| + +-----------------------+---------+-------+ ALTER TABLE StudentInfo ADD columns (LastName string, DOB timestamp); ---After Adding New columns to the table +-- After Adding New columns to the table DESC StudentInfo; -+--------------------------+------------+----------+--+ -| col_name | data_type | comment | -+--------------------------+------------+----------+--+ -| name | string | NULL | -| rollno | int | NULL | -| LastName | string | NULL | -| DOB | timestamp | NULL | -| age | int | NULL | -| # Partition Information | | | -| # col_name | data_type | comment | -| age | int | NULL | -+--------------------------+------------+----------+--+ + +-----------------------+---------+-------+ + | col_name|data_type|comment| + +-----------------------+---------+-------+ + | name| string| NULL| + | rollno| int| NULL| + | LastName| string| NULL| + | DOB|timestamp| NULL| + | age| int| NULL| + |# Partition Information| | | + | # col_name|data_type|comment| + | age| int| NULL| + +-----------------------+---------+-------+ -- Add a new partition to a table - SHOW PARTITIONS StudentInfo; -+------------+--+ -| partition | -+------------+--+ -| age=11 | -| age=12 | -| age=15 | -+------------+--+ + +---------+ + |partition| + +---------+ + | age=11| + | age=12| + | age=15| + +---------+ ALTER TABLE StudentInfo ADD IF NOT EXISTS PARTITION (age=18); -- After adding a new partition to the table SHOW PARTITIONS StudentInfo; -+------------+--+ -| partition | -+------------+--+ -| age=11 | -| age=12 | -| age=15 | -| age=18 | -+------------+--+ + +---------+ + |partition| + +---------+ + | age=11| + | age=12| + | age=15| + | age=18| + +---------+ -- Drop a partition from the table - SHOW PARTITIONS StudentInfo; -+------------+--+ -| partition | -+------------+--+ -| age=11 | -| age=12 | -| age=15 | -| age=18 | -+------------+--+ + +---------+ + |partition| + +---------+ + | age=11| + | age=12| + | age=15| + | age=18| + +---------+ ALTER TABLE StudentInfo DROP IF EXISTS PARTITION (age=18); -- After dropping the partition of the table SHOW PARTITIONS StudentInfo; -+------------+--+ -| partition | -+------------+--+ -| age=11 | -| age=12 | -| age=15 | -+------------+--+ + +---------+ + |partition| + +---------+ + | age=11| + | age=12| + | age=15| + +---------+ -- Adding multiple partitions to the table - SHOW PARTITIONS StudentInfo; -+------------+--+ -| partition | -+------------+--+ -| age=11 | -| age=12 | -| age=15 | -+------------+--+ + +---------+ + |partition| + +---------+ + | age=11| + | age=12| + | age=15| + +---------+ ALTER TABLE StudentInfo ADD IF NOT EXISTS PARTITION (age=18) PARTITION (age=20); -- After adding multiple partitions to the table SHOW PARTITIONS StudentInfo; -+------------+--+ -| partition | -+------------+--+ -| age=11 | -| age=12 | -| age=15 | -| age=18 | -| age=20 | -+------------+--+ + +---------+ + |partition| + +---------+ + | age=11| + | age=12| + | age=15| + | age=18| + | age=20| + +---------+ -- ALTER OR CHANGE COLUMNS - DESC StudentInfo; -+--------------------------+------------+----------+--+ -| col_name | data_type | comment | -+--------------------------+------------+----------+--+ -| name | string | NULL | -| rollno | int | NULL | -| LastName | string | NULL | -| DOB | timestamp | NULL | -| age | int | NULL | -| # Partition Information | | | -| # col_name | data_type | comment | -| age | int | NULL | -+--------------------------+------------+----------+--+ + +-----------------------+---------+-------+ + | col_name|data_type|comment| + +-----------------------+---------+-------+ + | name| string| NULL| + | rollno| int| NULL| + | LastName| string| NULL| + | DOB|timestamp| NULL| + | age| int| NULL| + |# Partition Information| | | + | # col_name|data_type|comment| + | age| int| NULL| + +-----------------------+---------+-------+ ALTER TABLE StudentInfo ALTER COLUMN name COMMENT "new comment"; --After ALTER or CHANGE COLUMNS DESC StudentInfo; -+--------------------------+------------+------------+--+ -| col_name | data_type | comment | -+--------------------------+------------+------------+--+ -| name | string | new comment| -| rollno | int | NULL | -| LastName | string | NULL | -| DOB | timestamp | NULL | -| age | int | NULL | -| # Partition Information | | | -| # col_name | data_type | comment | -| age | int | NULL | -+--------------------------+------------+------------+--+ - ---Change the fileformat + +-----------------------+---------+-----------+ + | col_name|data_type| comment| + +-----------------------+---------+-----------+ + | name| string|new comment| + | rollno| int| NULL| + | LastName| string| NULL| + | DOB|timestamp| NULL| + | age| int| NULL| + |# Partition Information| | | + | # col_name|data_type| comment| + | age| int| NULL| + +-----------------------+---------+-----------+ + +-- Change the fileformat ALTER TABLE loc_orc SET fileformat orc; ALTER TABLE p1 partition (month=2, day=2) SET fileformat parquet; ---Change the file Location +-- Change the file Location ALTER TABLE dbx.tab1 PARTITION (a='1', b='2') SET LOCATION '/path/to/part/ways' -- SET SERDE/ SERDE Properties @@ -469,17 +476,14 @@ ALTER TABLE test_tab SET SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinar ALTER TABLE dbx.tab1 SET SERDE 'org.apache.hadoop' WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee') ---SET TABLE PROPERTIES +-- SET TABLE PROPERTIES ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('winner' = 'loser') ---DROP TABLE PROPERTIES +-- DROP TABLE PROPERTIES ALTER TABLE dbx.tab1 UNSET TBLPROPERTIES ('winner') - {% endhighlight %} - ### Related Statements -- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) -- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) - + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) + * [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) diff --git a/docs/sql-ref-syntax-ddl-alter-view.md b/docs/sql-ref-syntax-ddl-alter-view.md index d42d040522cae..8116c97cc2f41 100644 --- a/docs/sql-ref-syntax-ddl-alter-view.md +++ b/docs/sql-ref-syntax-ddl-alter-view.md @@ -113,6 +113,8 @@ and the `view_identifier` must exist. ALTER VIEW view_identifier AS select_statement {% endhighlight %} +Note that `ALTER VIEW` statement does not support `SET SERDE` or `SET SERDEPROPERTIES` properties. + #### Parameters
    view_identifier
    @@ -139,97 +141,88 @@ ALTER VIEW tempdb1.v1 RENAME TO tempdb1.v2; -- Verify that the new view is created. DESCRIBE TABLE EXTENDED tempdb1.v2; - -+----------------------------+----------+-------+ -|col_name |data_type |comment| -+----------------------------+----------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -+----------------------------+----------+-------+ + +----------------------------+----------+-------+ + | col_name|data_type |comment| + +----------------------------+----------+-------+ + | c1| int| null| + | c2| string| null| + | | | | + |# Detailed Table Information| | | + | Database| tempdb1| | + | Table| v2| | + +----------------------------+----------+-------+ -- Before ALTER VIEW SET TBLPROPERTIES DESC TABLE EXTENDED tempdb1.v2; - -+----------------------------+----------+-------+ -|col_name |data_type |comment| -+----------------------------+----------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -|Table Properties |[....] | | -+----------------------------+----------+-------+ + +----------------------------+----------+-------+ + | col_name| data_type|comment| + +----------------------------+----------+-------+ + | c1| int| null| + | c2| string| null| + | | | | + |# Detailed Table Information| | | + | Database| tempdb1| | + | Table| v2| | + | Table Properties| [....]| | + +----------------------------+----------+-------+ -- Set properties in TBLPROPERTIES ALTER VIEW tempdb1.v2 SET TBLPROPERTIES ('created.by.user' = "John", 'created.date' = '01-01-2001' ); -- Use `DESCRIBE TABLE EXTENDED tempdb1.v2` to verify DESC TABLE EXTENDED tempdb1.v2; - -+----------------------------+-----------------------------------------------------+-------+ -|col_name |data_type |comment| -+----------------------------+-----------------------------------------------------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -|Table Properties |[created.by.user=John, created.date=01-01-2001, ....]| | -+----------------------------+-----------------------------------------------------+-------+ + +----------------------------+-----------------------------------------------------+-------+ + | col_name| data_type|comment| + +----------------------------+-----------------------------------------------------+-------+ + | c1| int| null| + | c2| string| null| + | | | | + |# Detailed Table Information| | | + | Database| tempdb1| | + | Table| v2| | + | Table Properties|[created.by.user=John, created.date=01-01-2001, ....]| | + +----------------------------+-----------------------------------------------------+-------+ -- Remove the key `created.by.user` and `created.date` from `TBLPROPERTIES` ALTER VIEW tempdb1.v2 UNSET TBLPROPERTIES ('created.by.user', 'created.date'); --Use `DESC TABLE EXTENDED tempdb1.v2` to verify the changes DESC TABLE EXTENDED tempdb1.v2; - -+----------------------------+----------+-------+ -|col_name |data_type |comment| -+----------------------------+----------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -|Table Properties |[....] | | -+----------------------------+----------+-------+ + +----------------------------+----------+-------+ + | col_name| data_type|comment| + +----------------------------+----------+-------+ + | c1| int| null| + | c2| string| null| + | | | | + |# Detailed Table Information| | | + | Database| tempdb1| | + | Table| v2| | + | Table Properties| [....]| | + +----------------------------+----------+-------+ -- Change the view definition ALTER VIEW tempdb1.v2 AS SELECT * FROM tempdb1.v1; -- Use `DESC TABLE EXTENDED` to verify DESC TABLE EXTENDED tempdb1.v2; - -+----------------------------+---------------------------+-------+ -|col_name |data_type |comment| -+----------------------------+---------------------------+-------+ -|c1 |int |null | -|c2 |string |null | -| | | | -|# Detailed Table Information| | | -|Database |tempdb1 | | -|Table |v2 | | -|Type |VIEW | | -|View Text |select * from tempdb1.v1 | | -|View Original Text |select * from tempdb1.v1 | | -+----------------------------+---------------------------+-------+ + +----------------------------+---------------------------+-------+ + | col_name| data_type|comment| + +----------------------------+---------------------------+-------+ + | c1| int| null| + | c2| string| null| + | | | | + |# Detailed Table Information| | | + | Database| tempdb1| | + | Table| v2| | + | Type| VIEW| | + | View Text| select * from tempdb1.v1| | + | View Original Text| select * from tempdb1.v1| | + +----------------------------+---------------------------+-------+ {% endhighlight %} ### Related Statements -- [describe-table](sql-ref-syntax-aux-describe-table.html) -- [create-view](sql-ref-syntax-ddl-create-view.html) -- [drop-view](sql-ref-syntax-ddl-drop-view.html) - -#### Note: - -`ALTER VIEW` statement does not support `SET SERDE` or `SET SERDEPROPERTIES` properties - + * [describe-table](sql-ref-syntax-aux-describe-table.html) + * [create-view](sql-ref-syntax-ddl-create-view.html) + * [drop-view](sql-ref-syntax-ddl-drop-view.html) + * [show-views](sql-ref-syntax-aux-show-views.html) diff --git a/docs/sql-ref-syntax-ddl-create-database.md b/docs/sql-ref-syntax-ddl-create-database.md index 4d2211c650953..6f74acdb60bf7 100644 --- a/docs/sql-ref-syntax-ddl-create-database.md +++ b/docs/sql-ref-syntax-ddl-create-database.md @@ -20,17 +20,20 @@ license: | --- ### Description + Creates a database with the specified name. If database with the same name already exists, an exception will be thrown. ### Syntax + {% highlight sql %} CREATE { DATABASE | SCHEMA } [ IF NOT EXISTS ] database_name - [ COMMENT database_comment ] - [ LOCATION database_directory ] - [ WITH DBPROPERTIES ( property_name = property_value [ , ... ] ) ] + [ COMMENT database_comment ] + [ LOCATION database_directory ] + [ WITH DBPROPERTIES ( property_name = property_value [ , ... ] ) ] {% endhighlight %} ### Parameters +
    database_name
    Specifies the name of the database to be created.
    @@ -49,6 +52,7 @@ CREATE { DATABASE | SCHEMA } [ IF NOT EXISTS ] database_name
    ### Examples + {% highlight sql %} -- Create database `customer_db`. This throws exception if database with name customer_db -- already exists. @@ -60,20 +64,21 @@ CREATE DATABASE IF NOT EXISTS customer_db; -- Create database `customer_db` only if database with same name doesn't exist with -- `Comments`,`Specific Location` and `Database properties`. CREATE DATABASE IF NOT EXISTS customer_db COMMENT 'This is customer database' LOCATION '/user' - WITH DBPROPERTIES (ID=001, Name='John'); + WITH DBPROPERTIES (ID=001, Name='John'); -- Verify that properties are set. DESCRIBE DATABASE EXTENDED customer_db; - +----------------------------+-----------------------------+ - | database_description_item | database_description_value | - +----------------------------+-----------------------------+ - | Database Name | customer_db | - | Description | This is customer database | - | Location | hdfs://hacluster/user | - | Properties | ((ID,001), (Name,John)) | - +----------------------------+-----------------------------+ + +-------------------------+--------------------------+ + |database_description_item|database_description_value| + +-------------------------+--------------------------+ + | Database Name| customer_db| + | Description| This is customer database| + | Location| hdfs://hacluster/user| + | Properties| ((ID,001), (Name,John))| + +-------------------------+--------------------------+ {% endhighlight %} ### Related Statements -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) -- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) + + * [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) + * [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) diff --git a/docs/sql-ref-syntax-ddl-create-function.md b/docs/sql-ref-syntax-ddl-create-function.md index 1f94bf6d25aa5..2bd26d18c7736 100644 --- a/docs/sql-ref-syntax-ddl-create-function.md +++ b/docs/sql-ref-syntax-ddl-create-function.md @@ -20,6 +20,7 @@ license: | --- ### Description + The `CREATE FUNCTION` statement is used to create a temporary or permanent function in Spark. Temporary functions are scoped at a session level where as permanent functions are created in the persistent catalog and are made available to @@ -31,12 +32,14 @@ aggregate functions using Scala, Python and Java APIs. Please refer to [aggregate functions](sql-getting-started#aggregations) for more information. ### Syntax + {% highlight sql %} CREATE [ OR REPLACE ] [ TEMPORARY ] FUNCTION [ IF NOT EXISTS ] function_name AS class_name [ resource_locations ] {% endhighlight %} ### Parameters +
    OR REPLACE
    @@ -90,6 +93,7 @@ CREATE [ OR REPLACE ] [ TEMPORARY ] FUNCTION [ IF NOT EXISTS ]
    ### Examples + {% highlight sql %} -- 1. Create a simple UDF `SimpleUdf` that increments the supplied integral value by 10. -- import org.apache.hadoop.hive.ql.exec.UDF; @@ -106,7 +110,7 @@ INSERT INTO test VALUES (1), (2); -- Create a permanent function called `simple_udf`. CREATE FUNCTION simple_udf AS 'SimpleUdf' - USING JAR '/tmp/SimpleUdf.jar'; + USING JAR '/tmp/SimpleUdf.jar'; -- Verify that the function is in the registry. SHOW USER FUNCTIONS; @@ -127,7 +131,7 @@ SELECT simple_udf(c1) AS function_return_value FROM t1; -- Created a temporary function. CREATE TEMPORARY FUNCTION simple_temp_udf AS 'SimpleUdf' - USING JAR '/tmp/SimpleUdf.jar'; + USING JAR '/tmp/SimpleUdf.jar'; -- Verify that the newly created temporary function is in the registry. -- Please note that the temporary function does not have a qualified @@ -152,20 +156,20 @@ SHOW USER FUNCTIONS; -- Replace the implementation of `simple_udf` CREATE OR REPLACE FUNCTION simple_udf AS 'SimpleUdfR' - USING JAR '/tmp/SimpleUdfR.jar'; + USING JAR '/tmp/SimpleUdfR.jar'; -- Invoke the function. Every selected value should be incremented by 20. SELECT simple_udf(c1) AS function_return_value FROM t1; -+---------------------+ -|function_return_value| -+---------------------+ -| 21| -| 22| -+---------------------+ - + +---------------------+ + |function_return_value| + +---------------------+ + | 21| + | 22| + +---------------------+ {% endhighlight %} -### Related statements -- [SHOW FUNCTIONS](sql-ref-syntax-aux-show-functions.html) -- [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) -- [DROP FUNCTION](sql-ref-syntax-ddl-drop-function.html) +### Related Statements + + * [SHOW FUNCTIONS](sql-ref-syntax-aux-show-functions.html) + * [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) + * [DROP FUNCTION](sql-ref-syntax-ddl-drop-function.html) diff --git a/docs/sql-ref-syntax-ddl-create-table-datasource.md b/docs/sql-ref-syntax-ddl-create-table-datasource.md index 532377d7fcec3..715b64c33baed 100644 --- a/docs/sql-ref-syntax-ddl-create-table-datasource.md +++ b/docs/sql-ref-syntax-ddl-create-table-datasource.md @@ -24,19 +24,20 @@ license: | The `CREATE TABLE` statement defines a new table using a Data Source. ### Syntax + {% highlight sql %} CREATE TABLE [ IF NOT EXISTS ] table_identifier - [ ( col_name1 col_type1 [ COMMENT col_comment1 ], ... ) ] - [USING data_source] - [ OPTIONS ( key1=val1, key2=val2, ... ) ] - [ PARTITIONED BY ( col_name1, col_name2, ... ) ] - [ CLUSTERED BY ( col_name3, col_name4, ... ) - [ SORTED BY ( col_name [ ASC | DESC ], ... ) ] - INTO num_buckets BUCKETS ] - [ LOCATION path ] - [ COMMENT table_comment ] - [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] - [ AS select_statement ] + [ ( col_name1 col_type1 [ COMMENT col_comment1 ], ... ) ] + [USING data_source] + [ OPTIONS ( key1=val1, key2=val2, ... ) ] + [ PARTITIONED BY ( col_name1, col_name2, ... ) ] + [ CLUSTERED BY ( col_name3, col_name4, ... ) + [ SORTED BY ( col_name [ ASC | DESC ], ... ) ] + INTO num_buckets BUCKETS ] + [ LOCATION path ] + [ COMMENT table_comment ] + [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] + [ AS select_statement ] {% endhighlight %} Note that, the clauses between the USING clause and the AS SELECT clause can come in @@ -95,6 +96,7 @@ as any order. For example, you can write COMMENT table_comment after TBLPROPERTI
    ### Data Source Interaction + A Data Source table acts like a pointer to the underlying data source. For example, you can create a table "foo" in Spark which points to a table "bar" in MySQL using JDBC Data Source. When you read/write table "foo", you actually read/write table "bar". @@ -107,6 +109,7 @@ For CREATE TABLE AS SELECT, Spark will overwrite the underlying data source with input query, to make sure the table gets created contains exactly the same data as the input query. ### Examples + {% highlight sql %} --Use data source @@ -114,29 +117,29 @@ CREATE TABLE student (id INT, name STRING, age INT) USING CSV; --Use data from another table CREATE TABLE student_copy USING CSV - AS SELECT * FROM student; + AS SELECT * FROM student; --Omit the USING clause, which uses the default data source (parquet by default) CREATE TABLE student (id INT, name STRING, age INT); --Specify table comment and properties CREATE TABLE student (id INT, name STRING, age INT) USING CSV - COMMENT 'this is a comment' - TBLPROPERTIES ('foo'='bar'); + COMMENT 'this is a comment' + TBLPROPERTIES ('foo'='bar'); --Specify table comment and properties with different clauses order CREATE TABLE student (id INT, name STRING, age INT) USING CSV - TBLPROPERTIES ('foo'='bar') - COMMENT 'this is a comment'; + TBLPROPERTIES ('foo'='bar') + COMMENT 'this is a comment'; --Create partitioned and bucketed table CREATE TABLE student (id INT, name STRING, age INT) - USING CSV - PARTITIONED BY (age) - CLUSTERED BY (Id) INTO 4 buckets; - + USING CSV + PARTITIONED BY (age) + CLUSTERED BY (Id) INTO 4 buckets; {% endhighlight %} ### Related Statements -* [CREATE TABLE USING HIVE FORMAT](sql-ref-syntax-ddl-create-table-hiveformat.html) -* [CREATE TABLE LIKE](sql-ref-syntax-ddl-create-table-like.html) + + * [CREATE TABLE USING HIVE FORMAT](sql-ref-syntax-ddl-create-table-hiveformat.html) + * [CREATE TABLE LIKE](sql-ref-syntax-ddl-create-table-like.html) diff --git a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md index 0425bafd94398..06f353ad2f103 100644 --- a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md +++ b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md @@ -18,23 +18,24 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description The `CREATE TABLE` statement defines a new table using Hive format. ### Syntax + {% highlight sql %} CREATE [ EXTERNAL ] TABLE [ IF NOT EXISTS ] table_identifier - [ ( col_name1[:] col_type1 [ COMMENT col_comment1 ], ... ) ] - [ COMMENT table_comment ] - [ PARTITIONED BY ( col_name2[:] col_type2 [ COMMENT col_comment2 ], ... ) - | ( col_name1, col_name2, ... ) ] - [ ROW FORMAT row_format ] - [ STORED AS file_format ] - [ LOCATION path ] - [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] - [ AS select_statement ] - + [ ( col_name1[:] col_type1 [ COMMENT col_comment1 ], ... ) ] + [ COMMENT table_comment ] + [ PARTITIONED BY ( col_name2[:] col_type2 [ COMMENT col_comment2 ], ... ) + | ( col_name1, col_name2, ... ) ] + [ ROW FORMAT row_format ] + [ STORED AS file_format ] + [ LOCATION path ] + [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] + [ AS select_statement ] {% endhighlight %} Note that, the clauses between the columns definition clause and the AS SELECT clause can come in @@ -93,47 +94,45 @@ as any order. For example, you can write COMMENT table_comment after TBLPROPERTI
    The table is populated using the data from the select statement.
    - ### Examples -{% highlight sql %} +{% highlight sql %} --Use hive format CREATE TABLE student (id INT, name STRING, age INT) STORED AS ORC; --Use data from another table CREATE TABLE student_copy STORED AS ORC - AS SELECT * FROM student; + AS SELECT * FROM student; --Specify table comment and properties CREATE TABLE student (id INT, name STRING, age INT) - COMMENT 'this is a comment' - STORED AS ORC - TBLPROPERTIES ('foo'='bar'); + COMMENT 'this is a comment' + STORED AS ORC + TBLPROPERTIES ('foo'='bar'); --Specify table comment and properties with different clauses order CREATE TABLE student (id INT, name STRING, age INT) - STORED AS ORC - TBLPROPERTIES ('foo'='bar') - COMMENT 'this is a comment'; + STORED AS ORC + TBLPROPERTIES ('foo'='bar') + COMMENT 'this is a comment'; --Create partitioned table CREATE TABLE student (id INT, name STRING) - PARTITIONED BY (age INT) - STORED AS ORC; + PARTITIONED BY (age INT) + STORED AS ORC; --Create partitioned table with different clauses order CREATE TABLE student (id INT, name STRING) - STORED AS ORC - PARTITIONED BY (age INT); + STORED AS ORC + PARTITIONED BY (age INT); --Use Row Format and file format CREATE TABLE student (id INT,name STRING) - ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' - STORED AS TEXTFILE; - + ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + STORED AS TEXTFILE; {% endhighlight %} - ### Related Statements -* [CREATE TABLE USING DATASOURCE](sql-ref-syntax-ddl-create-table-datasource.html) -* [CREATE TABLE LIKE](sql-ref-syntax-ddl-create-table-like.html) + + * [CREATE TABLE USING DATASOURCE](sql-ref-syntax-ddl-create-table-datasource.html) + * [CREATE TABLE LIKE](sql-ref-syntax-ddl-create-table-like.html) diff --git a/docs/sql-ref-syntax-ddl-create-table-like.md b/docs/sql-ref-syntax-ddl-create-table-like.md index f49fd7fb24c91..fe1dc4b1ef258 100644 --- a/docs/sql-ref-syntax-ddl-create-table-like.md +++ b/docs/sql-ref-syntax-ddl-create-table-like.md @@ -18,21 +18,24 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + ### Description The `CREATE TABLE` statement defines a new table using the definition/metadata of an existing table or view. ### Syntax + {% highlight sql %} CREATE TABLE [IF NOT EXISTS] table_identifier LIKE source_table_identifier -USING data_source -[ ROW FORMAT row_format ] -[ STORED AS file_format ] -[ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] -[ LOCATION path ] + USING data_source + [ ROW FORMAT row_format ] + [ STORED AS file_format ] + [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] + [ LOCATION path ] {% endhighlight %} ### Parameters +
    table_identifier
    @@ -70,28 +73,27 @@ USING data_source
    Path to the directory where table data is stored,Path to the directory where table data is stored, which could be a path on distributed storage like HDFS, etc. Location to create an external table.
    - ### Examples -{% highlight sql %} ---Create table using an existing table +{% highlight sql %} +-- Create table using an existing table CREATE TABLE Student_Dupli like Student; ---Create table like using a data source +-- Create table like using a data source CREATE TABLE Student_Dupli like Student USING CSV; ---Table is created as external table at the location specified +-- Table is created as external table at the location specified CREATE TABLE Student_Dupli like Student location '/root1/home'; ---Create table like using a rowformat +-- Create table like using a rowformat CREATE TABLE Student_Dupli like Student - ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' - STORED AS TEXTFILE - TBLPROPERTIES ('owner'='xxxx'); - + ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + STORED AS TEXTFILE + TBLPROPERTIES ('owner'='xxxx'); {% endhighlight %} ### Related Statements -* [CREATE TABLE USING DATASOURCE](sql-ref-syntax-ddl-create-table-datasource.html) -* [CREATE TABLE USING HIVE FORMAT](sql-ref-syntax-ddl-create-table-hiveformat.html) + + * [CREATE TABLE USING DATASOURCE](sql-ref-syntax-ddl-create-table-datasource.html) + * [CREATE TABLE USING HIVE FORMAT](sql-ref-syntax-ddl-create-table-hiveformat.html) diff --git a/docs/sql-ref-syntax-ddl-create-table.md b/docs/sql-ref-syntax-ddl-create-table.md index 20aff6fb823cb..b0388adbc9a38 100644 --- a/docs/sql-ref-syntax-ddl-create-table.md +++ b/docs/sql-ref-syntax-ddl-create-table.md @@ -20,13 +20,16 @@ license: | --- ### Description + `CREATE TABLE` statement is used to define a table in an existing database. The CREATE statements: -* [CREATE TABLE USING DATA_SOURCE](sql-ref-syntax-ddl-create-table-datasource.html) -* [CREATE TABLE USING HIVE FORMAT](sql-ref-syntax-ddl-create-table-hiveformat.html) -* [CREATE TABLE LIKE](sql-ref-syntax-ddl-create-table-like.html) + + * [CREATE TABLE USING DATA_SOURCE](sql-ref-syntax-ddl-create-table-datasource.html) + * [CREATE TABLE USING HIVE FORMAT](sql-ref-syntax-ddl-create-table-hiveformat.html) + * [CREATE TABLE LIKE](sql-ref-syntax-ddl-create-table-like.html) ### Related Statements -- [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) -- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) + + * [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) + * [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md index 67060d70f0790..5d9db2ab36b5d 100644 --- a/docs/sql-ref-syntax-ddl-create-view.md +++ b/docs/sql-ref-syntax-ddl-create-view.md @@ -20,17 +20,20 @@ license: | --- ### Description + Views are based on the result-set of an `SQL` query. `CREATE VIEW` constructs a virtual table that has no physical data therefore other operations like `ALTER VIEW` and `DROP VIEW` only change metadata. ### Syntax + {% highlight sql %} CREATE [ OR REPLACE ] [ [ GLOBAL ] TEMPORARY ] VIEW [ IF NOT EXISTS ] view_identifier create_view_clauses AS query {% endhighlight %} ### Parameters +
    OR REPLACE
    If a view of same name already exists, it will be replaced.
    @@ -71,6 +74,7 @@ CREATE [ OR REPLACE ] [ [ GLOBAL ] TEMPORARY ] VIEW [ IF NOT EXISTS ] view_ident
    ### Examples + {% highlight sql %} -- Create or replace view for `experienced_employee` with comments. CREATE OR REPLACE VIEW experienced_employee @@ -87,5 +91,7 @@ CREATE GLOBAL TEMPORARY VIEW IF NOT EXISTS subscribed_movies {% endhighlight %} ### Related Statements -- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) -- [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) + + * [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) + * [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) + * [SHOW VIEWS](sql-ref-syntax-aux-show-views.html) diff --git a/docs/sql-ref-syntax-ddl-drop-database.md b/docs/sql-ref-syntax-ddl-drop-database.md index 0bdb98f2b129c..431139101eba4 100644 --- a/docs/sql-ref-syntax-ddl-drop-database.md +++ b/docs/sql-ref-syntax-ddl-drop-database.md @@ -30,7 +30,6 @@ exception will be thrown if the database does not exist in the system. DROP ( DATABASE | SCHEMA ) [ IF EXISTS ] dbname [ RESTRICT | CASCADE ] {% endhighlight %} - ### Parameters
    @@ -54,27 +53,20 @@ DROP ( DATABASE | SCHEMA ) [ IF EXISTS ] dbname [ RESTRICT | CASCADE ]
    ### Example + {% highlight sql %} -- Create `inventory_db` Database CREATE DATABASE inventory_db COMMENT 'This database is used to maintain Inventory'; -- Drop the database and it's tables DROP DATABASE inventory_db CASCADE; -+---------+ -| Result | -+---------+ -+---------+ -- Drop the database using IF EXISTS DROP DATABASE IF EXISTS inventory_db CASCADE; -+---------+ -| Result | -+---------+ -+---------+ - {% endhighlight %} -### Related statements -- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) -- [SHOW DATABASES](sql-ref-syntax-aux-show-databases.html) +### Related Statements + + * [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) + * [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) + * [SHOW DATABASES](sql-ref-syntax-aux-show-databases.html) diff --git a/docs/sql-ref-syntax-ddl-drop-function.md b/docs/sql-ref-syntax-ddl-drop-function.md index 16d08d1ae8e99..f7ad18553e304 100644 --- a/docs/sql-ref-syntax-ddl-drop-function.md +++ b/docs/sql-ref-syntax-ddl-drop-function.md @@ -20,15 +20,16 @@ license: | --- ### Description + The `DROP FUNCTION` statement drops a temporary or user defined function (UDF). An exception will - be thrown if the function does not exist. +be thrown if the function does not exist. ### Syntax + {% highlight sql %} DROP [ TEMPORARY ] FUNCTION [ IF EXISTS ] [ db_name. ] function_name {% endhighlight %} - ### Parameters
    @@ -47,36 +48,33 @@ DROP [ TEMPORARY ] FUNCTION [ IF EXISTS ] [ db_name. ] function_name
    ### Example + {% highlight sql %} -- Create a permanent function `test_avg` CREATE FUNCTION test_avg as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'; -- List user functions SHOW USER FUNCTIONS; - +-------------------+ - | function | - +-------------------+ - | default.test_avg | - +-------------------+ + +----------------+ + | function| + +----------------+ + |default.test_avg| + +----------------+ -- Create Temporary function `test_avg` CREATE TEMPORARY FUNCTION test_avg as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'; -- List user functions SHOW USER FUNCTIONS; - +-------------------+ - | function | - +-------------------+ - | default.test_avg | - | test_avg | - +-------------------+ + +----------------+ + | function| + +----------------+ + |default.test_avg| + | test_avg| + +----------------+ -- Drop Permanent function DROP FUNCTION test_avg; - +---------+ - | Result | - +---------+ - +---------+ -- Try to drop Permanent function which is not present DROP FUNCTION test_avg; @@ -86,20 +84,18 @@ DROP FUNCTION test_avg; -- List the functions after dropping, it should list only temporary function SHOW USER FUNCTIONS; - +-----------+ - | function | - +-----------+ - | test_avg | - +-----------+ + +--------+ + |function| + +--------+ + |test_avg| + +--------+ -- Drop Temporary function DROP TEMPORARY FUNCTION IF EXISTS test_avg; - +---------+ - | Result | - +---------+ - +---------+ {% endhighlight %} -### Related statements -- [CREATE FUNCTION](sql-ref-syntax-ddl-create-function.html) -- [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) -- [SHOW FUNCTION](sql-ref-syntax-aux-show-functions.html) + +### Related Statements + + * [CREATE FUNCTION](sql-ref-syntax-ddl-create-function.html) + * [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) + * [SHOW FUNCTION](sql-ref-syntax-aux-show-functions.html) diff --git a/docs/sql-ref-syntax-ddl-drop-table.md b/docs/sql-ref-syntax-ddl-drop-table.md index d1d8534efe7a2..32a9cc7bb27db 100644 --- a/docs/sql-ref-syntax-ddl-drop-table.md +++ b/docs/sql-ref-syntax-ddl-drop-table.md @@ -27,11 +27,13 @@ if the table is not `EXTERNAL` table. If the table is not present it throws an e In case of an external table, only the associated metadata information is removed from the metastore database. ### Syntax + {% highlight sql %} DROP TABLE [ IF EXISTS ] table_identifier {% endhighlight %} ### Parameter +
    IF EXISTS
    @@ -48,40 +50,27 @@ DROP TABLE [ IF EXISTS ] table_identifier
    ### Example + {% highlight sql %} -- Assumes a table named `employeetable` exists. DROP TABLE employeetable; -+---------+--+ -| Result | -+---------+--+ -+---------+--+ -- Assumes a table named `employeetable` exists in the `userdb` database DROP TABLE userdb.employeetable; -+---------+--+ -| Result | -+---------+--+ -+---------+--+ -- Assumes a table named `employeetable` does not exists. -- Throws exception DROP TABLE employeetable; -Error: org.apache.spark.sql.AnalysisException: Table or view not found: employeetable; -(state=,code=0) + Error: org.apache.spark.sql.AnalysisException: Table or view not found: employeetable; + (state=,code=0) -- Assumes a table named `employeetable` does not exists,Try with IF EXISTS -- this time it will not throw exception DROP TABLE IF EXISTS employeetable; -+---------+--+ -| Result | -+---------+--+ -+---------+--+ - {% endhighlight %} ### Related Statements -- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) -- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) -- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) - + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) + * [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) + * [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) diff --git a/docs/sql-ref-syntax-ddl-drop-view.md b/docs/sql-ref-syntax-ddl-drop-view.md index bf9e497b3a418..ae976c125f5f8 100644 --- a/docs/sql-ref-syntax-ddl-drop-view.md +++ b/docs/sql-ref-syntax-ddl-drop-view.md @@ -20,14 +20,17 @@ license: | --- ### Description + `DROP VIEW` removes the metadata associated with a specified view from the catalog. ### Syntax + {% highlight sql %} DROP VIEW [ IF EXISTS ] view_identifier {% endhighlight %} ### Parameter +
    IF EXISTS
    @@ -44,39 +47,29 @@ DROP VIEW [ IF EXISTS ] view_identifier
    ### Example + {% highlight sql %} -- Assumes a view named `employeeView` exists. DROP VIEW employeeView; -+---------+--+ -| Result | -+---------+--+ -+---------+--+ -- Assumes a view named `employeeView` exists in the `userdb` database DROP VIEW userdb.employeeView; -+---------+--+ -| Result | -+---------+--+ -+---------+--+ -- Assumes a view named `employeeView` does not exists. -- Throws exception DROP VIEW employeeView; -Error: org.apache.spark.sql.AnalysisException: Table or view not found: employeeView; -(state=,code=0) + Error: org.apache.spark.sql.AnalysisException: Table or view not found: employeeView; + (state=,code=0) -- Assumes a view named `employeeView` does not exists,Try with IF EXISTS -- this time it will not throw exception DROP VIEW IF EXISTS employeeView; -+---------+--+ -| Result | -+---------+--+ -+---------+--+ - {% endhighlight %} ### Related Statements -- [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) -- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) -- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) -- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) + + * [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) + * [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) + * [SHOW VIEWS](sql-ref-syntax-aux-show-views.html) + * [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) + * [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) diff --git a/docs/sql-ref-syntax-ddl-repair-table.md b/docs/sql-ref-syntax-ddl-repair-table.md index daa6a46fca58f..499b2bff54d8d 100644 --- a/docs/sql-ref-syntax-ddl-repair-table.md +++ b/docs/sql-ref-syntax-ddl-repair-table.md @@ -20,14 +20,17 @@ license: | --- ### Description + `MSCK REPAIR TABLE` recovers all the partitions in the directory of a table and updates the Hive metastore. When creating a table using `PARTITIONED BY` clause, partitions are generated and registered in the Hive metastore. However, if the partitioned table is created from existing data, partitions are not registered automatically in the Hive metastore. User needs to run `MSCK REPAIR TABLE` to register the partitions. `MSCK REPAIR TABLE` on a non-existent table or a table without partitions throws an exception. Another way to recover partitions is to use `ALTER TABLE RECOVER PARTITIONS`. ### Syntax + {% highlight sql %} MSCK REPAIR TABLE table_identifier {% endhighlight %} ### Parameters +
    table_identifier
    @@ -40,30 +43,31 @@ MSCK REPAIR TABLE table_identifier
    ### Examples -{% highlight sql %} - -- create a partitioned table from existing data /tmp/namesAndAges.parquet - CREATE TABLE t1 (name STRING, age INT) USING parquet PARTITIONED BY (age) - location "/tmp/namesAndAges.parquet"; - -- SELECT * FROM t1 does not return results - SELECT * FROM t1; - - -- run MSCK REPAIR TABLE to recovers all the partitions - MSCK REPAIR TABLE t1; +{% highlight sql %} +-- create a partitioned table from existing data /tmp/namesAndAges.parquet +CREATE TABLE t1 (name STRING, age INT) USING parquet PARTITIONED BY (age) + LOCATION "/tmp/namesAndAges.parquet"; - -- SELECT * FROM t1 returns results - SELECT * FROM t1; +-- SELECT * FROM t1 does not return results +SELECT * FROM t1; - + -------------- + ------+ - | name | age | - + -------------- + ------+ - | Michael | 20 | - + -------------- + ------+ - | Justin | 19 | - + -------------- + ----- + - | Andy | 30 | - + -------------- + ----- + +-- run MSCK REPAIR TABLE to recovers all the partitions +MSCK REPAIR TABLE t1; +-- SELECT * FROM t1 returns results +SELECT * FROM t1; + +-------+---+ + | name|age| + +-------+---+ + |Michael| 20| + +-------+---+ + | Justin| 19| + +-------+---+ + | Andy| 30| + +-------+---+ {% endhighlight %} + ### Related Statements + * [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) diff --git a/docs/sql-ref-syntax-ddl-truncate-table.md b/docs/sql-ref-syntax-ddl-truncate-table.md index 3a0569e809d84..6377e83570207 100644 --- a/docs/sql-ref-syntax-ddl-truncate-table.md +++ b/docs/sql-ref-syntax-ddl-truncate-table.md @@ -20,16 +20,19 @@ license: | --- ### Description + The `TRUNCATE TABLE` statement removes all the rows from a table or partition(s). The table must not be a view or an external/temporary table. In order to truncate multiple partitions at once, the user can specify the partitions in `partition_spec`. If no `partition_spec` is specified it will remove all partitions in the table. ### Syntax + {% highlight sql %} TRUNCATE TABLE table_identifier [ partition_spec ] {% endhighlight %} ### Parameters +
    table_identifier
    @@ -52,47 +55,43 @@ TRUNCATE TABLE table_identifier [ partition_spec ]
    - ### Examples -{% highlight sql %} ---Create table Student with partition -CREATE TABLE Student ( name String, rollno INT) PARTITIONED BY (age int); +{% highlight sql %} +-- Create table Student with partition +CREATE TABLE Student (name STRING, rollno INT) PARTITIONED BY (age INT); SELECT * from Student; -+-------+---------+------+--+ -| name | rollno | age | -+-------+---------+------+--+ -| ABC | 1 | 10 | -| DEF | 2 | 10 | -| XYZ | 3 | 12 | -+-------+---------+------+--+ + +----+------+---+ + |name|rollno|age| + +----+------+---+ + | ABC| 1| 10| + | DEF| 2| 10| + | XYZ| 3| 12| + +----+------+---+ -- Removes all rows from the table in the partition specified TRUNCATE TABLE Student partition(age=10); ---After truncate execution, records belonging to partition age=10 are removed +-- After truncate execution, records belonging to partition age=10 are removed SELECT * from Student; -+-------+---------+------+--+ -| name | rollno | age | -+-------+---------+------+--+ -| XYZ | 3 | 12 | -+-------+---------+------+--+ + +----+------+---+ + |name|rollno|age| + +----+------+---+ + | XYZ| 3| 12| + +----+------+---+ -- Removes all rows from the table from all partitions TRUNCATE TABLE Student; SELECT * from Student; -+-------+---------+------+--+ -| name | rollno | age | -+-------+---------+------+--+ -+-------+---------+------+--+ -No rows selected - + +----+------+---+ + |name|rollno|age| + +----+------+---+ + +----+------+---+ {% endhighlight %} - ### Related Statements -- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) -- [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) + * [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) + * [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) diff --git a/docs/sql-ref-syntax-ddl.md b/docs/sql-ref-syntax-ddl.md index 954020a874455..82fbf0498a20f 100644 --- a/docs/sql-ref-syntax-ddl.md +++ b/docs/sql-ref-syntax-ddl.md @@ -19,20 +19,19 @@ license: | limitations under the License. --- - Data Definition Statements are used to create or modify the structure of database objects in a database. Spark SQL supports the following Data Definition Statements: - -- [ALTER DATABASE](sql-ref-syntax-ddl-alter-database.html) -- [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) -- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) -- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) -- [CREATE FUNCTION](sql-ref-syntax-ddl-create-function.html) -- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) -- [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) -- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) -- [DROP FUNCTION](sql-ref-syntax-ddl-drop-function.html) -- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) -- [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) -- [TRUNCATE TABLE](sql-ref-syntax-ddl-truncate-table.html) -- [REPAIR TABLE](sql-ref-syntax-ddl-repair-table.html) + * [ALTER DATABASE](sql-ref-syntax-ddl-alter-database.html) + * [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) + * [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) + * [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) + * [CREATE FUNCTION](sql-ref-syntax-ddl-create-function.html) + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) + * [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) + * [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) + * [DROP FUNCTION](sql-ref-syntax-ddl-drop-function.html) + * [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) + * [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) + * [TRUNCATE TABLE](sql-ref-syntax-ddl-truncate-table.html) + * [REPAIR TABLE](sql-ref-syntax-ddl-repair-table.html) + * [USE DATABASE](sql-ref-syntax-qry-select-usedb.html) diff --git a/docs/sql-ref-syntax-dml-insert-into.md b/docs/sql-ref-syntax-dml-insert-into.md index 715f43c9b80ea..ba65334ef8f61 100644 --- a/docs/sql-ref-syntax-dml-insert-into.md +++ b/docs/sql-ref-syntax-dml-insert-into.md @@ -24,12 +24,14 @@ license: | The `INSERT INTO` statement inserts new rows into a table. The inserted rows can be specified by value expressions or result from a query. ### Syntax + {% highlight sql %} INSERT INTO [ TABLE ] table_identifier [ partition_spec ] { { VALUES ( { value | NULL } [ , ... ] ) [ , ( ... ) ] } | query } {% endhighlight %} ### Parameters +
    table_identifier
    @@ -70,149 +72,148 @@ INSERT INTO [ TABLE ] table_identifier [ partition_spec ]
    ### Examples -#### Single Row Insert Using a VALUES Clause -{% highlight sql %} - CREATE TABLE students (name VARCHAR(64), address VARCHAR(64), student_id INT) - USING PARQUET PARTITIONED BY (student_id); - - INSERT INTO students - VALUES ('Amy Smith', '123 Park Ave, San Jose', 111111); - SELECT * FROM students; +#### Single Row Insert Using a VALUES Clause - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Amy Smith | 123 Park Ave, San Jose | 111111 | - + -------------- + ------------------------------ + -------------- + +{% highlight sql %} +CREATE TABLE students (name VARCHAR(64), address VARCHAR(64), student_id INT) + USING PARQUET PARTITIONED BY (student_id); + +INSERT INTO students VALUES + ('Amy Smith', '123 Park Ave, San Jose', 111111); + +SELECT * FROM students; + +---------+---------------------+----------+ + | name| address|student_id| + +---------+---------------------+----------+ + |Amy Smith|123 Park Ave,San Jose| 111111| + +---------+---------------------+----------+ {% endhighlight %} #### Multi-Row Insert Using a VALUES Clause + {% highlight sql %} - INSERT INTO students - VALUES ('Bob Brown', '456 Taylor St, Cupertino', 222222), - ('Cathy Johnson', '789 Race Ave, Palo Alto', 333333); - - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Amy Smith | 123 Park Ave, San Jose | 111111 | - + -------------- + ------------------------------ + -------------- + - | Bob Brown | 456 Taylor St, Cupertino | 222222 | - + -------------- + ------------------------------ + -------------- + - | Cathy Johnson | 789 Race Ave, Palo Alto | 333333 | - + -------------- + ------------------------------ + -------------- + +INSERT INTO students VALUES + ('Bob Brown', '456 Taylor St, Cupertino', 222222), + ('Cathy Johnson', '789 Race Ave, Palo Alto', 333333); + +SELECT * FROM students; + +-------------+------------------------+----------+ + | name| address|student_id| + +-------------+------------------------+----------+ + | Amy Smith| 123 Park Ave, San Jose| 111111| + +-------------+------------------------+----------+ + | Bob Brown|456 Taylor St, Cupertino| 222222| + +-------------+------------------------+----------+ + |Cathy Johnson| 789 Race Ave, Palo Alto| 333333| + +--------------+-----------------------+----------+ {% endhighlight %} #### Insert Using a SELECT Statement + {% highlight sql %} - -- Assuming the persons table has already been created and populated. - SELECT * FROM persons; - - + -------------- + ------------------------------ + -------------- + - | name | address | ssn | - + -------------- + ------------------------------ + -------------- + - | Dora Williams | 134 Forest Ave, Melo Park | 123456789 | - + -------------- + ------------------------------ + -------------- + - | Eddie Davis | 245 Market St, Milpitas | 345678901 | - + -------------- + ------------------------------ + ---------------+ - - INSERT INTO students PARTITION (student_id = 444444) - SELECT name, address FROM persons WHERE name = "Dora Williams"; - - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Amy Smith | 123 Park Ave, San Jose | 111111 | - + -------------- + ------------------------------ + -------------- + - | Bob Brown | 456 Taylor St, Cupertino | 222222 | - + -------------- + ------------------------------ + -------------- + - | Cathy Johnson | 789 Race Ave, Palo Alto | 333333 | - + -------------- + ------------------------------ + -------------- + - | Dora Williams | 134 Forest Ave, Melo Park | 444444 | - + -------------- + ------------------------------ + -------------- + +-- Assuming the persons table has already been created and populated. +SELECT * FROM persons; + +-------------+-------------------------+---------+ + | name| address| ssn| + +-------------+-------------------------+---------+ + |Dora Williams|134 Forest Ave, Melo Park|123456789| + +-------------+-------------------------+---------+ + | Eddie Davis| 245 Market St, Milpitas|345678901| + +-------------+-------------------------+---------+ + +INSERT INTO students PARTITION (student_id = 444444) + SELECT name, address FROM persons WHERE name = "Dora Williams"; + +SELECT * FROM students; + +-------------+-------------------------+----------+ + | name| address|student_id| + +-------------+-------------------------+----------+ + | Amy Smith| 123 Park Ave, San Jose| 111111| + +-------------+-------------------------+----------+ + | Bob Brown| 456 Taylor St, Cupertino| 222222| + +-------------+-------------------------+----------+ + |Cathy Johnson| 789 Race Ave, Palo Alto| 333333| + +-------------+-------------------------+----------+ + |Dora Williams|134 Forest Ave, Melo Park| 444444| + +-------------+-------------------------+----------+ {% endhighlight %} #### Insert Using a TABLE Statement + {% highlight sql %} - -- Assuming the visiting_students table has already been created and populated. - SELECT * FROM visiting_students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Fleur Laurent | 345 Copper St, London | 777777 | - + -------------- + ------------------------------ + -------------- + - | Gordon Martin | 779 Lake Ave, Oxford | 888888 | - + -------------- + ------------------------------ + -------------- + - - INSERT INTO students TABLE visiting_students; - - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Amy Smith | 123 Park Ave, San Jose | 111111 | - + -------------- + ------------------------------ + -------------- + - | Bob Brown | 456 Taylor St, Cupertino | 222222 | - + -------------- + ------------------------------ + -------------- + - | Cathy Johnson | 789 Race Ave, Palo Alto | 333333 | - + -------------- + ------------------------------ + -------------- + - | Dora Williams | 134 Forest Ave, Melo Park | 444444 | - + -------------- + ------------------------------ + -------------- + - | Fleur Laurent | 345 Copper St, London | 777777 | - + -------------- + ------------------------------ + -------------- + - | Gordon Martin | 779 Lake Ave, Oxford | 888888 | - + -------------- + ------------------------------ + -------------- + +-- Assuming the visiting_students table has already been created and populated. +SELECT * FROM visiting_students; + +-------------+---------------------+----------+ + | name| address|student_id| + +-------------+---------------------+----------+ + |Fleur Laurent|345 Copper St, London| 777777| + +-------------+---------------------+----------+ + |Gordon Martin| 779 Lake Ave, Oxford| 888888| + +-------------+---------------------+----------+ + +INSERT INTO students TABLE visiting_students; + +SELECT * FROM students; + +-------------+-------------------------+----------+ + | name| address|student_id| + +-------------+-------------------------+----------+ + | Amy Smith| 123 Park Ave,San Jose| 111111| + +-------------+-------------------------+----------+ + | Bob Brown| 456 Taylor St, Cupertino| 222222| + +-------------+-------------------------+----------+ + |Cathy Johnson| 789 Race Ave, Palo Alto| 333333| + +-------------+-------------------------+----------+ + |Dora Williams|134 Forest Ave, Melo Park| 444444| + +-------------+-------------------------+----------+ + |Fleur Laurent| 345 Copper St, London| 777777| + +-------------+-------------------------+----------+ + |Gordon Martin| 779 Lake Ave, Oxford| 888888| + +-------------+-------------------------+----------+ {% endhighlight %} #### Insert Using a FROM Statement + {% highlight sql %} - -- Assuming the applicants table has already been created and populated. - SELECT * FROM applicants; - - + -------------- + ------------------------------ + -------------- + -------------- + - | name | address | student_id | qualified | - + -------------- + ------------------------------ + -------------- + -------------- + - | Helen Davis | 469 Mission St, San Diego | 999999 | true | - + -------------- + ------------------------------ + -------------- + -------------- + - | Ivy King | 367 Leigh Ave, Santa Clara | 101010 | false | - + -------------- + ------------------------------ + -------------- + -------------- + - | Jason Wang | 908 Bird St, Saratoga | 121212 | true | - + -------------- + ------------------------------ + -------------- + -------------- + - - INSERT INTO students - FROM applicants SELECT name, address, id applicants WHERE qualified = true; - - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Amy Smith | 123 Park Ave, San Jose | 111111 | - + -------------- + ------------------------------ + -------------- + - | Bob Brown | 456 Taylor St, Cupertino | 222222 | - + -------------- + ------------------------------ + -------------- + - | Cathy Johnson | 789 Race Ave, Palo Alto | 333333 | - + -------------- + ------------------------------ + -------------- + - | Dora Williams | 134 Forest Ave, Melo Park | 444444 | - + -------------- + ------------------------------ + -------------- + - | Fleur Laurent | 345 Copper St, London | 777777 | - + -------------- + ------------------------------ + -------------- + - | Gordon Martin | 779 Lake Ave, Oxford | 888888 | - + -------------- + ------------------------------ + -------------- + - | Helen Davis | 469 Mission St, San Diego | 999999 | - + -------------- + ------------------------------ + -------------- + - | Jason Wang | 908 Bird St, Saratoga | 121212 | - + -------------- + ------------------------------ + -------------- + +-- Assuming the applicants table has already been created and populated. +SELECT * FROM applicants; + +-----------+--------------------------+----------+---------+ + | name| address|student_id|qualified| + +-----------+--------------------------+----------+---------+ + |Helen Davis| 469 Mission St, San Diego| 999999| true| + +-----------+--------------------------+----------+---------+ + | Ivy King|367 Leigh Ave, Santa Clara| 101010| false| + +-----------+--------------------------+----------+---------+ + | Jason Wang| 908 Bird St, Saratoga| 121212| true| + +-----------+--------------------------+----------+---------+ + +INSERT INTO students + FROM applicants SELECT name, address, id applicants WHERE qualified = true; + +SELECT * FROM students; + +-------------+-------------------------+----------+ + | name| address|student_id| + +-------------+-------------------------+----------+ + | Amy Smith| 123 Park Ave, San Jose| 111111| + +-------------+-------------------------+----------+ + | Bob Brown| 456 Taylor St, Cupertino| 222222| + +-------------+-------------------------+----------+ + |Cathy Johnson| 789 Race Ave, Palo Alto| 333333| + +-------------+-------------------------+----------+ + |Dora Williams|134 Forest Ave, Melo Park| 444444| + +-------------+-------------------------+----------+ + |Fleur Laurent| 345 Copper St, London| 777777| + +-------------+-------------------------+----------+ + |Gordon Martin| 779 Lake Ave, Oxford| 888888| + +-------------+-------------------------+----------+ + | Helen Davis|469 Mission St, San Diego| 999999| + +-------------+-------------------------+----------+ + | Jason Wang| 908 Bird St, Saratoga| 121212| + +-------------+-------------------------+----------+ {% endhighlight %} ### Related Statements - * [INSERT OVERWRITE statement](sql-ref-syntax-dml-insert-overwrite-table.html) - * [INSERT OVERWRITE DIRECTORY statement](sql-ref-syntax-dml-insert-overwrite-directory.html) - * [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) + + * [INSERT OVERWRITE statement](sql-ref-syntax-dml-insert-overwrite-table.html) + * [INSERT OVERWRITE DIRECTORY statement](sql-ref-syntax-dml-insert-overwrite-directory.html) + * [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) diff --git a/docs/sql-ref-syntax-dml-insert-overwrite-directory-hive.md b/docs/sql-ref-syntax-dml-insert-overwrite-directory-hive.md index 3b0475aef1015..3ab0994cf06e7 100644 --- a/docs/sql-ref-syntax-dml-insert-overwrite-directory-hive.md +++ b/docs/sql-ref-syntax-dml-insert-overwrite-directory-hive.md @@ -20,17 +20,20 @@ license: | --- ### Description + The `INSERT OVERWRITE DIRECTORY` with Hive format overwrites the existing data in the directory with the new values using Hive `SerDe`. Hive support must be enabled to use this command. The inserted rows can be specified by value expressions or result from a query. ### Syntax + {% highlight sql %} INSERT OVERWRITE [ LOCAL ] DIRECTORY directory_path - [ ROW FORMAT row_format ] [ STORED AS file_format ] - { { VALUES ( { value | NULL } [ , ... ] ) [ , ( ... ) ] } | query } + [ ROW FORMAT row_format ] [ STORED AS file_format ] + { { VALUES ( { value | NULL } [ , ... ] ) [ , ( ... ) ] } | query } {% endhighlight %} ### Parameters +
    directory_path
    @@ -71,17 +74,19 @@ INSERT OVERWRITE [ LOCAL ] DIRECTORY directory_path
    ### Examples + {% highlight sql %} - INSERT OVERWRITE LOCAL DIRECTORY '/tmp/destination' - STORED AS orc - SELECT * FROM test_table; +INSERT OVERWRITE LOCAL DIRECTORY '/tmp/destination' + STORED AS orc + SELECT * FROM test_table; - INSERT OVERWRITE LOCAL DIRECTORY '/tmp/destination' - ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' - SELECT * FROM test_table; +INSERT OVERWRITE LOCAL DIRECTORY '/tmp/destination' + ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + SELECT * FROM test_table; {% endhighlight %} ### Related Statements - * [INSERT INTO statement](sql-ref-syntax-dml-insert-into.html) - * [INSERT OVERWRITE statement](sql-ref-syntax-dml-insert-overwrite-table.html) - * [INSERT OVERWRITE DIRECTORY statement](sql-ref-syntax-dml-insert-overwrite-directory.html) + + * [INSERT INTO statement](sql-ref-syntax-dml-insert-into.html) + * [INSERT OVERWRITE statement](sql-ref-syntax-dml-insert-overwrite-table.html) + * [INSERT OVERWRITE DIRECTORY statement](sql-ref-syntax-dml-insert-overwrite-directory.html) diff --git a/docs/sql-ref-syntax-dml-insert-overwrite-directory.md b/docs/sql-ref-syntax-dml-insert-overwrite-directory.md index 6f19c62bc3d56..645396620f21a 100644 --- a/docs/sql-ref-syntax-dml-insert-overwrite-directory.md +++ b/docs/sql-ref-syntax-dml-insert-overwrite-directory.md @@ -9,19 +9,22 @@ license: | 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. --- + ### Description -The `INSERT OVERWRITE DIRECTORY` statement overwrites the existing data in the directory with the new values using Spark native format. The inserted rows can be specified by value expressions or result from a query. + +The `INSERT OVERWRITE DIRECTORY` statement overwrites the existing data in the directory with the new values using a given Spark file format. The inserted rows can be specified by value expressions or result from a query. ### Syntax + {% highlight sql %} INSERT OVERWRITE [ LOCAL ] DIRECTORY [ directory_path ] USING file_format [ OPTIONS ( key = val [ , ... ] ) ] @@ -29,6 +32,7 @@ INSERT OVERWRITE [ LOCAL ] DIRECTORY [ directory_path ] {% endhighlight %} ### Parameters +
    directory_path
    @@ -39,13 +43,13 @@ INSERT OVERWRITE [ LOCAL ] DIRECTORY [ directory_path ]
    file_format
    - Specifies the file format to use for the insert. Valid options are TEXT, CSV, JSON, JDBC, PARQUET, ORC, HIVE, DELTA, LIBSVM, or a fully qualified class name of a custom implementation of org.apache.spark.sql.sources.DataSourceRegister. + Specifies the file format to use for the insert. Valid options are TEXT, CSV, JSON, JDBC, PARQUET, ORC, HIVE, LIBSVM, or a fully qualified class name of a custom implementation of org.apache.spark.sql.execution.datasources.FileFormat.
    OPTIONS ( key = val [ , ... ] )
    -
    Specifies one or more table property key and value pairs.
    +
    Specifies one or more options for the writing of the file format.
    @@ -67,6 +71,7 @@ INSERT OVERWRITE [ LOCAL ] DIRECTORY [ directory_path ]
    ### Examples + {% highlight sql %} INSERT OVERWRITE DIRECTORY '/tmp/destination' USING parquet @@ -80,6 +85,7 @@ INSERT OVERWRITE DIRECTORY {% endhighlight %} ### Related Statements - * [INSERT INTO statement](sql-ref-syntax-dml-insert-into.html) - * [INSERT OVERWRITE statement](sql-ref-syntax-dml-insert-overwrite-table.html) - * [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) + + * [INSERT INTO statement](sql-ref-syntax-dml-insert-into.html) + * [INSERT OVERWRITE statement](sql-ref-syntax-dml-insert-overwrite-table.html) + * [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) diff --git a/docs/sql-ref-syntax-dml-insert-overwrite-table.md b/docs/sql-ref-syntax-dml-insert-overwrite-table.md index 1f32342642b2a..added8e1976be 100644 --- a/docs/sql-ref-syntax-dml-insert-overwrite-table.md +++ b/docs/sql-ref-syntax-dml-insert-overwrite-table.md @@ -9,9 +9,9 @@ license: | 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. @@ -24,12 +24,14 @@ license: | The `INSERT OVERWRITE` statement overwrites the existing data in the table using the new values. The inserted rows can be specified by value expressions or result from a query. ### Syntax + {% highlight sql %} INSERT OVERWRITE [ TABLE ] table_identifier [ partition_spec [ IF NOT EXISTS ] ] { { VALUES ( { value | NULL } [ , ... ] ) [ , ( ... ) ] } | query } {% endhighlight %} ### Parameters +
    table_identifier
    @@ -70,129 +72,120 @@ INSERT OVERWRITE [ TABLE ] table_identifier [ partition_spec [ IF NOT EXISTS ] ]
    ### Examples + #### Insert Using a VALUES Clause + {% highlight sql %} - -- Assuming the students table has already been created and populated. - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Amy Smith | 123 Park Ave, San Jose | 111111 | - + -------------- + ------------------------------ + -------------- + - | Bob Brown | 456 Taylor St, Cupertino | 222222 | - + -------------- + ------------------------------ + -------------- + - | Cathy Johnson | 789 Race Ave, Palo Alto | 333333 | - + -------------- + ------------------------------ + -------------- + - | Dora Williams | 134 Forest Ave, Melo Park | 444444 | - + -------------- + ------------------------------ + -------------- + - | Fleur Laurent | 345 Copper St, London | 777777 | - + -------------- + ------------------------------ + -------------- + - | Gordon Martin | 779 Lake Ave, Oxford | 888888 | - + -------------- + ------------------------------ + -------------- + - | Helen Davis | 469 Mission St, San Diego | 999999 | - + -------------- + ------------------------------ + -------------- + - | Jason Wang | 908 Bird St, Saratoga | 121212 | - + -------------- + ------------------------------ + -------------- + - - INSERT OVERWRITE students - VALUES ('Ashua Hill', '456 Erica Ct, Cupertino', 111111), - ('Brian Reed', '723 Kern Ave, Palo Alto', 222222); - - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Ashua Hill | 456 Erica Ct, Cupertino | 111111 | - + -------------- + ------------------------------ + -------------- + - | Brian Reed | 723 Kern Ave, Palo Alto | 222222 | - + -------------- + ------------------------------ + -------------- + +-- Assuming the students table has already been created and populated. +SELECT * FROM students; + +-------------+-------------------------+----------+ + | name| address|student_id| + +-------------+-------------------------+----------+ + | Amy Smith| 123 Park Ave, San Jose| 111111| + | Bob Brown| 456 Taylor St, Cupertino| 222222| + |Cathy Johnson| 789 Race Ave, Palo Alto| 333333| + |Dora Williams|134 Forest Ave, Melo Park| 444444| + |Fleur Laurent| 345 Copper St, London| 777777| + |Gordon Martin| 779 Lake Ave, Oxford| 888888| + | Helen Davis|469 Mission St, San Diego| 999999| + | Jason Wang| 908 Bird St, Saratoga| 121212| + +-------------+-------------------------+----------+ + +INSERT OVERWRITE students VALUES + ('Ashua Hill', '456 Erica Ct, Cupertino', 111111), + ('Brian Reed', '723 Kern Ave, Palo Alto', 222222); + +SELECT * FROM students; + +----------+-----------------------+----------+ + | name| address|student_id| + +----------+-----------------------+----------+ + |Ashua Hill|456 Erica Ct, Cupertino| 111111| + |Brian Reed|723 Kern Ave, Palo Alto| 222222| + +----------+-----------------------+----------+ + {% endhighlight %} #### Insert Using a SELECT Statement + {% highlight sql %} - -- Assuming the persons table has already been created and populated. - SELECT * FROM persons; - - + -------------- + ------------------------------ + -------------- + - | name | address | ssn | - + -------------- + ------------------------------ + -------------- + - | Dora Williams | 134 Forest Ave, Melo Park | 123456789 | - + -------------- + ------------------------------ + -------------- + - | Eddie Davis | 245 Market St, Milpitas | 345678901 | - + -------------- + ------------------------------ + ---------------+ - - INSERT OVERWRITE students PARTITION (student_id = 222222) - SELECT name, address FROM persons WHERE name = "Dora Williams"; - - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Ashua Hill | 456 Erica Ct, Cupertino | 111111 | - + -------------- + ------------------------------ + -------------- + - | Dora Williams | 134 Forest Ave, Melo Park | 222222 | - + -------------- + ------------------------------ + -------------- + +-- Assuming the persons table has already been created and populated. +SELECT * FROM persons; + +-------------+-------------------------+---------+ + | name| address| ssn| + +-------------+-------------------------+---------+ + |Dora Williams|134 Forest Ave, Melo Park|123456789| + +-------------+-------------------------+---------+ + | Eddie Davis| 245 Market St,Milpitas|345678901| + +-------------+-------------------------+---------+ + +INSERT OVERWRITE students PARTITION (student_id = 222222) + SELECT name, address FROM persons WHERE name = "Dora Williams"; + +SELECT * FROM students; + +-------------+-------------------------+----------+ + | name| address|student_id| + +-------------+-------------------------+----------+ + | Ashua Hill| 456 Erica Ct, Cupertino| 111111| + +-------------+-------------------------+----------+ + |Dora Williams|134 Forest Ave, Melo Park| 222222| + +-------------+-------------------------+----------+ {% endhighlight %} #### Insert Using a TABLE Statement + {% highlight sql %} - -- Assuming the visiting_students table has already been created and populated. - SELECT * FROM visiting_students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Fleur Laurent | 345 Copper St, London | 777777 | - + -------------- + ------------------------------ + -------------- + - | Gordon Martin | 779 Lake Ave, Oxford | 888888 | - + -------------- + ------------------------------ + -------------- + - - INSERT OVERWRITE students TABLE visiting_students; - - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Fleur Laurent | 345 Copper St, London | 777777 | - + -------------- + ------------------------------ + -------------- + - | Gordon Martin | 779 Lake Ave, Oxford | 888888 | - + -------------- + ------------------------------ + -------------- + +-- Assuming the visiting_students table has already been created and populated. +SELECT * FROM visiting_students; + +-------------+---------------------+----------+ + | name| address|student_id| + +-------------+---------------------+----------+ + |Fleur Laurent|345 Copper St, London| 777777| + +-------------+---------------------+----------+ + |Gordon Martin| 779 Lake Ave, Oxford| 888888| + +-------------+---------------------+----------+ + +INSERT OVERWRITE students TABLE visiting_students; + +SELECT * FROM students; + +-------------+---------------------+----------+ + | name| address|student_id| + +-------------+---------------------+----------+ + |Fleur Laurent|345 Copper St, London| 777777| + +-------------+---------------------+----------+ + |Gordon Martin| 779 Lake Ave, Oxford| 888888| + +-------------+---------------------+----------+ {% endhighlight %} #### Insert Using a FROM Statement + {% highlight sql %} - -- Assuming the applicants table has already been created and populated. - SELECT * FROM applicants; - - + -------------- + ------------------------------ + -------------- + -------------- + - | name | address | student_id | qualified | - + -------------- + ------------------------------ + -------------- + -------------- + - | Helen Davis | 469 Mission St, San Diego | 999999 | true | - + -------------- + ------------------------------ + -------------- + -------------- + - | Ivy King | 367 Leigh Ave, Santa Clara | 101010 | false | - + -------------- + ------------------------------ + -------------- + -------------- + - | Jason Wang | 908 Bird St, Saratoga | 121212 | true | - + -------------- + ------------------------------ + -------------- + -------------- + - - INSERT OVERWRITE students; - FROM applicants SELECT name, address, id applicants WHERE qualified = true; - - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Helen Davis | 469 Mission St, San Diego | 999999 | - + -------------- + ------------------------------ + -------------- + - | Jason Wang | 908 Bird St, Saratoga | 121212 | - + -------------- + ------------------------------ + -------------- + +-- Assuming the applicants table has already been created and populated. +SELECT * FROM applicants; + +-----------+--------------------------+----------+---------+ + | name| address|student_id|qualified| + +-----------+--------------------------+----------+---------+ + |Helen Davis| 469 Mission St, San Diego| 999999| true| + +-----------+--------------------------+----------+---------+ + | Ivy King|367 Leigh Ave, Santa Clara| 101010| false| + +-----------+--------------------------+----------+---------+ + | Jason Wang| 908 Bird St, Saratoga| 121212| true| + +-----------+--------------------------+----------+---------+ + +INSERT OVERWRITE students + FROM applicants SELECT name, address, id applicants WHERE qualified = true; + +SELECT * FROM students; + +-----------+-------------------------+----------+ + | name| address|student_id| + +-----------+-------------------------+----------+ + |Helen Davis|469 Mission St, San Diego| 999999| + +-----------+-------------------------+----------+ + | Jason Wang| 908 Bird St, Saratoga| 121212| + +-----------+-------------------------+----------+ {% endhighlight %} ### Related Statements - * [INSERT INTO statement](sql-ref-syntax-dml-insert-into.html) - * [INSERT OVERWRITE DIRECTORY statement](sql-ref-syntax-dml-insert-overwrite-directory.html) - * [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) + + * [INSERT INTO statement](sql-ref-syntax-dml-insert-into.html) + * [INSERT OVERWRITE DIRECTORY statement](sql-ref-syntax-dml-insert-overwrite-directory.html) + * [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) diff --git a/docs/sql-ref-syntax-dml-insert.md b/docs/sql-ref-syntax-dml-insert.md index 15a2e28896943..2345add2460c8 100644 --- a/docs/sql-ref-syntax-dml-insert.md +++ b/docs/sql-ref-syntax-dml-insert.md @@ -20,7 +20,8 @@ license: | --- The INSERT statements: -* [INSERT INTO statement](sql-ref-syntax-dml-insert-into.html) -* [INSERT OVERWRITE statement](sql-ref-syntax-dml-insert-overwrite-table.html) -* [INSERT OVERWRITE DIRECTORY statement](sql-ref-syntax-dml-insert-overwrite-directory.html) -* [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) + + * [INSERT INTO statement](sql-ref-syntax-dml-insert-into.html) + * [INSERT OVERWRITE statement](sql-ref-syntax-dml-insert-overwrite-table.html) + * [INSERT OVERWRITE DIRECTORY statement](sql-ref-syntax-dml-insert-overwrite-directory.html) + * [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) diff --git a/docs/sql-ref-syntax-dml-load.md b/docs/sql-ref-syntax-dml-load.md index aadfd708e274e..9a9bf230e3101 100644 --- a/docs/sql-ref-syntax-dml-load.md +++ b/docs/sql-ref-syntax-dml-load.md @@ -9,9 +9,9 @@ license: | 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. @@ -20,14 +20,17 @@ license: | --- ### Description -`LOAD DATA` statement loads the data into a table from the user specified directory or file. If a directory is specified then all the files from the directory are loaded. If a file is specified then only the single file is loaded. Additionally the `LOAD DATA` statement takes an optional partition specification. When a partition is specified, the data files (when input source is a directory) or the single file (when input source is a file) are loaded into the partition of the target table. + +`LOAD DATA` statement loads the data into a Hive serde table from the user specified directory or file. If a directory is specified then all the files from the directory are loaded. If a file is specified then only the single file is loaded. Additionally the `LOAD DATA` statement takes an optional partition specification. When a partition is specified, the data files (when input source is a directory) or the single file (when input source is a file) are loaded into the partition of the target table. ### Syntax + {% highlight sql %} LOAD DATA [ LOCAL ] INPATH path [ OVERWRITE ] INTO TABLE table_identifier [ partition_spec ] {% endhighlight %} ### Parameters +
    path
    Path of the file system. It can be either an absolute or a relative path.
    @@ -67,65 +70,57 @@ LOAD DATA [ LOCAL ] INPATH path [ OVERWRITE ] INTO TABLE table_identifier [ part
    ### Examples -{% highlight sql %} - -- Example without partition specification. - -- Assuming the students table has already been created and populated. - SELECT * FROM students; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Amy Smith | 123 Park Ave, San Jose | 111111 | - + -------------- + ------------------------------ + -------------- + - - CREATE TABLE test_load (name VARCHAR(64), address VARCHAR(64), student_id INT); - - -- Assuming the students table is in '/user/hive/warehouse/' - LOAD DATA LOCAL INPATH '/user/hive/warehouse/students' OVERWRITE INTO TABLE test_load; - - SELECT * FROM test_load; - - + -------------- + ------------------------------ + -------------- + - | name | address | student_id | - + -------------- + ------------------------------ + -------------- + - | Amy Smith | 123 Park Ave, San Jose | 111111 | - + -------------- + ------------------------------ + -------------- + - - -- Example with partition specification. - CREATE TABLE test_partition (c1 INT, c2 INT, c3 INT) USING HIVE PARTITIONED BY (c2, c3); - - INSERT INTO test_partition PARTITION (c2 = 2, c3 = 3) VALUES (1); - - INSERT INTO test_partition PARTITION (c2 = 5, c3 = 6) VALUES (4); - - INSERT INTO test_partition PARTITION (c2 = 8, c3 = 9) VALUES (7); - - SELECT * FROM test_partition; - - + ------- + ------- + ----- + - | c1 | c2 | c3 | - + ------- + --------------- + - | 1 | 2 | 3 | - + ------- + ------- + ----- + - | 4 | 5 | 6 | - + ------- + ------- + ----- + - | 7 | 8 | 9 | - + ------- + ------- + ----- + - - CREATE TABLE test_load_partition (c1 INT, c2 INT, c3 INT) USING HIVE PARTITIONED BY (c2, c3); - - -- Assuming the test_partition table is in '/user/hive/warehouse/' - LOAD DATA LOCAL INPATH '/user/hive/warehouse/test_partition/c2=2/c3=3' - OVERWRITE INTO TABLE test_load_partition PARTITION (c2=2, c3=3); - - SELECT * FROM test_load_partition; - - + ------- + ------- + ----- + - | c1 | c2 | c3 | - + ------- + --------------- + - | 1 | 2 | 3 | - + ------- + ------- + ----- + - +{% highlight sql %} +-- Example without partition specification. +-- Assuming the students table has already been created and populated. +SELECT * FROM students; + +---------+----------------------+----------+ + | name| address|student_id| + +---------+----------------------+----------+ + |Amy Smith|123 Park Ave, San Jose| 111111| + +---------+----------------------+----------+ + +CREATE TABLE test_load (name VARCHAR(64), address VARCHAR(64), student_id INT) USING HIVE; + +-- Assuming the students table is in '/user/hive/warehouse/' +LOAD DATA LOCAL INPATH '/user/hive/warehouse/students' OVERWRITE INTO TABLE test_load; + +SELECT * FROM test_load; + +---------+----------------------+----------+ + | name| address|student_id| + +---------+----------------------+----------+ + |Amy Smith|123 Park Ave, San Jose| 111111| + +---------+----------------------+----------+ + +-- Example with partition specification. +CREATE TABLE test_partition (c1 INT, c2 INT, c3 INT) PARTITIONED BY (c2, c3); + +INSERT INTO test_partition PARTITION (c2 = 2, c3 = 3) VALUES (1); + +INSERT INTO test_partition PARTITION (c2 = 5, c3 = 6) VALUES (4); + +INSERT INTO test_partition PARTITION (c2 = 8, c3 = 9) VALUES (7); + +SELECT * FROM test_partition; + +---+---+---+ + | c1| c2| c3| + +---+---+---+ + | 1| 2| 3| + | 4| 5| 6| + | 7| 8| 9| + +---+---+---+ + +CREATE TABLE test_load_partition (c1 INT, c2 INT, c3 INT) USING HIVE PARTITIONED BY (c2, c3); + +-- Assuming the test_partition table is in '/user/hive/warehouse/' +LOAD DATA LOCAL INPATH '/user/hive/warehouse/test_partition/c2=2/c3=3' + OVERWRITE INTO TABLE test_load_partition PARTITION (c2=2, c3=3); + +SELECT * FROM test_load_partition; + +---+---+---+ + | c1| c2| c3| + +---+---+---+ + | 1| 2| 3| + +---+---+---+ {% endhighlight %} - diff --git a/docs/sql-ref-syntax-dml.md b/docs/sql-ref-syntax-dml.md index b5dd45f8962c9..9f75990555f64 100644 --- a/docs/sql-ref-syntax-dml.md +++ b/docs/sql-ref-syntax-dml.md @@ -21,5 +21,5 @@ license: | Data Manipulation Statements are used to add, change, or delete data. Spark SQL supports the following Data Manipulation Statements: -- [INSERT](sql-ref-syntax-dml-insert.html) -- [LOAD](sql-ref-syntax-dml-load.html) + * [INSERT](sql-ref-syntax-dml-insert.html) + * [LOAD](sql-ref-syntax-dml-load.html) diff --git a/docs/sql-ref-syntax-qry-explain.md b/docs/sql-ref-syntax-qry-explain.md index 7e18e16bc8ea6..6a7c2ace8223b 100644 --- a/docs/sql-ref-syntax-qry-explain.md +++ b/docs/sql-ref-syntax-qry-explain.md @@ -24,8 +24,8 @@ license: | The `EXPLAIN` statement is used to provide logical/physical plans for an input statement. By default, this clause provides information about a physical plan only. - ### Syntax + {% highlight sql %} EXPLAIN [EXTENDED | CODEGEN | COST | FORMATTED] statement {% endhighlight %} @@ -64,76 +64,72 @@ EXPLAIN [EXTENDED | CODEGEN | COST | FORMATTED] statement
    ### Examples -{% highlight sql %} - ---Default Output +{% highlight sql %} +-- Default Output EXPLAIN select k, sum(v) from values (1, 2), (1, 3) t(k, v) group by k; -+----------------------------------------------------+ -| plan | -+----------------------------------------------------+ -| == Physical Plan == -*(2) HashAggregate(keys=[k#33], functions=[sum(cast(v#34 as bigint))]) -+- Exchange hashpartitioning(k#33, 200), true, [id=#59] - +- *(1) HashAggregate(keys=[k#33], functions=[partial_sum(cast(v#34 as bigint))]) - +- *(1) LocalTableScan [k#33, v#34] -| - +---------------------------------------------------- + +----------------------------------------------------+ + | plan| + +----------------------------------------------------+ + | == Physical Plan == + *(2) HashAggregate(keys=[k#33], functions=[sum(cast(v#34 as bigint))]) + +- Exchange hashpartitioning(k#33, 200), true, [id=#59] + +- *(1) HashAggregate(keys=[k#33], functions=[partial_sum(cast(v#34 as bigint))]) + +- *(1) LocalTableScan [k#33, v#34] + | + +---------------------------------------------------- -- Using Extended - EXPLAIN EXTENDED select k, sum(v) from values (1, 2), (1, 3) t(k, v) group by k; -+----------------------------------------------------+ -| plan | -+----------------------------------------------------+ -| == Parsed Logical Plan == -'Aggregate ['k], ['k, unresolvedalias('sum('v), None)] -+- 'SubqueryAlias `t` - +- 'UnresolvedInlineTable [k, v], [List(1, 2), List(1, 3)] - -== Analyzed Logical Plan == -k: int, sum(v): bigint -Aggregate [k#47], [k#47, sum(cast(v#48 as bigint)) AS sum(v)#50L] -+- SubqueryAlias `t` + +----------------------------------------------------+ + | plan| + +----------------------------------------------------+ + | == Parsed Logical Plan == + 'Aggregate ['k], ['k, unresolvedalias('sum('v), None)] + +- 'SubqueryAlias `t` + +- 'UnresolvedInlineTable [k, v], [List(1, 2), List(1, 3)] + + == Analyzed Logical Plan == + k: int, sum(v): bigint + Aggregate [k#47], [k#47, sum(cast(v#48 as bigint)) AS sum(v)#50L] + +- SubqueryAlias `t` + +- LocalRelation [k#47, v#48] + + == Optimized Logical Plan == + Aggregate [k#47], [k#47, sum(cast(v#48 as bigint)) AS sum(v)#50L] +- LocalRelation [k#47, v#48] - -== Optimized Logical Plan == -Aggregate [k#47], [k#47, sum(cast(v#48 as bigint)) AS sum(v)#50L] -+- LocalRelation [k#47, v#48] - -== Physical Plan == -*(2) HashAggregate(keys=[k#47], functions=[sum(cast(v#48 as bigint))], output=[k#47, sum(v)#50L]) -+- Exchange hashpartitioning(k#47, 200), true, [id=#79] - +- *(1) HashAggregate(keys=[k#47], functions=[partial_sum(cast(v#48 as bigint))], output=[k#47, sum#52L]) + + == Physical Plan == + *(2) HashAggregate(keys=[k#47], functions=[sum(cast(v#48 as bigint))], output=[k#47, sum(v)#50L]) + +- Exchange hashpartitioning(k#47, 200), true, [id=#79] + +- *(1) HashAggregate(keys=[k#47], functions=[partial_sum(cast(v#48 as bigint))], output=[k#47, sum#52L]) +- *(1) LocalTableScan [k#47, v#48] - | -+----------------------------------------------------+ - ---Using Formatted + | + +----------------------------------------------------+ +-- Using Formatted EXPLAIN FORMATTED select k, sum(v) from values (1, 2), (1, 3) t(k, v) group by k; -+----------------------------------------------------+ -| plan | -+----------------------------------------------------+ -| == Physical Plan == -* HashAggregate (4) -+- Exchange (3) - +- * HashAggregate (2) - +- * LocalTableScan (1) - - -(1) LocalTableScan [codegen id : 1] -Output: [k#19, v#20] - -(2) HashAggregate [codegen id : 1] -Input: [k#19, v#20] - -(3) Exchange -Input: [k#19, sum#24L] - -(4) HashAggregate [codegen id : 2] -Input: [k#19, sum#24L] - | -+----------------------------------------------------+ - + +----------------------------------------------------+ + | plan| + +----------------------------------------------------+ + | == Physical Plan == + * HashAggregate (4) + +- Exchange (3) + +- * HashAggregate (2) + +- * LocalTableScan (1) + + + (1) LocalTableScan [codegen id : 1] + Output: [k#19, v#20] + + (2) HashAggregate [codegen id : 1] + Input: [k#19, v#20] + + (3) Exchange + Input: [k#19, sum#24L] + + (4) HashAggregate [codegen id : 2] + Input: [k#19, sum#24L] + | + +----------------------------------------------------+ {% endhighlight %} diff --git a/docs/sql-ref-syntax-qry-sampling.md b/docs/sql-ref-syntax-qry-sampling.md index a5efb361ee847..3bc45cc48b78f 100644 --- a/docs/sql-ref-syntax-qry-sampling.md +++ b/docs/sql-ref-syntax-qry-sampling.md @@ -19,4 +19,76 @@ license: | limitations under the License. --- -**This page is under construction** +### Description + +The `TABLESAMPLE` statement is used to sample the table. It supports the following sampling methods: + * `TABLESAMPLE`(x `ROWS`): Sample the table down to the given number of rows. + * `TABLESAMPLE`(x `PERCENT`): Sample the table down to the given percentage. Note that percentages are defined as a number between 0 and 100. + * `TABLESAMPLE`(`BUCKET` x `OUT OF` y): Sample the table down to a `x` out of `y` fraction. + +Note: `TABLESAMPLE` returns the approximate number of rows or fraction requested. + +### Syntax + +{% highlight sql %} +TABLESAMPLE ((integer_expression | decimal_expression) PERCENT) + | TABLESAMPLE (integer_expression ROWS) + | TABLESAMPLE (BUCKET integer_expression OUT OF integer_expression) +{% endhighlight %} + +### Examples + +{% highlight sql %} +SELECT * FROM test; + +--+----+ + |id|name| + +--+----+ + | 5|Alex| + | 8|Lucy| + | 2|Mary| + | 4|Fred| + | 1|Lisa| + | 9|Eric| + |10|Adam| + | 6|Mark| + | 7|Lily| + | 3|Evan| + +--+----+ + +SELECT * FROM test TABLESAMPLE (50 PERCENT); + +--+----+ + |id|name| + +--+----+ + | 5|Alex| + | 2|Mary| + | 4|Fred| + | 9|Eric| + |10|Adam| + | 3|Evan| + +--+----+ + +SELECT * FROM test TABLESAMPLE (5 ROWS); + +--+----+ + |id|name| + +--+----+ + | 5|Alex| + | 8|Lucy| + | 2|Mary| + | 4|Fred| + | 1|Lisa| + +--+----+ + +SELECT * FROM test TABLESAMPLE (BUCKET 4 OUT OF 10); + +--+----+ + |id|name| + +--+----+ + | 8|Lucy| + | 2|Mary| + | 9|Eric| + | 6|Mark| + +--+----+ +{% endhighlight %} + +### Related Statement + + * [SELECT](sql-ref-syntax-qry-select.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry-select-clusterby.md b/docs/sql-ref-syntax-qry-select-clusterby.md index 8f1dc59806f80..687b2b512cd90 100644 --- a/docs/sql-ref-syntax-qry-select-clusterby.md +++ b/docs/sql-ref-syntax-qry-select-clusterby.md @@ -18,6 +18,9 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + +### Description + The CLUSTER BY clause is used to first repartition the data based on the input expressions and then sort the data within each partition. This is semantically equivalent to performing a @@ -26,11 +29,13 @@ semantically equivalent to performing a resultant rows are sorted within each partition and does not guarantee a total order of output. ### Syntax + {% highlight sql %} CLUSTER BY { expression [ , ... ] } {% endhighlight %} ### Parameters +
    expression
    @@ -39,6 +44,7 @@ CLUSTER BY { expression [ , ... ] }
    ### Examples + {% highlight sql %} CREATE TABLE person (name STRING, age INT); INSERT INTO person VALUES @@ -58,16 +64,15 @@ SET spark.sql.shuffle.partitions = 2; -- of a query when `CLUSTER BY` is not used vs when it's used. The query below produces rows -- where age column is not sorted. SELECT age, name FROM person; - +---+-------+ - |age|name | + |age| name| +---+-------+ - |16 |Shone S| - |25 |Zen Hui| - |16 |Jack N | - |25 |Mike A | - |18 |John A | - |18 |Anil B | + | 16|Shone S| + | 25|Zen Hui| + | 16| Jack N| + | 25| Mike A| + | 18| John A| + | 18| Anil B| +---+-------+ -- Produces rows clustered by age. Persons with same age are clustered together. @@ -75,25 +80,25 @@ SELECT age, name FROM person; -- persons with age 16 are in the second partition. The rows are sorted based -- on age within each partition. SELECT age, name FROM person CLUSTER BY age; - +---+-------+ - |age|name | + |age| name| +---+-------+ - |18 |John A | - |18 |Anil B | - |25 |Zen Hui| - |25 |Mike A | - |16 |Shone S| - |16 |Jack N | + | 18| John A| + | 18| Anil B| + | 25|Zen Hui| + | 25| Mike A| + | 16|Shone S| + | 16| Jack N| +---+-------+ {% endhighlight %} -### Related Clauses -- [SELECT Main](sql-ref-syntax-qry-select.html) -- [WHERE Clause](sql-ref-syntax-qry-select-where.html) -- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) -- [HAVING Clause](sql-ref-syntax-qry-select-having.html) -- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) -- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) -- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) -- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +### Related Statements + + * [SELECT Main](sql-ref-syntax-qry-select.html) + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select-distribute-by.md b/docs/sql-ref-syntax-qry-select-distribute-by.md index 957df9c48eee7..18d73c7cdff19 100644 --- a/docs/sql-ref-syntax-qry-select-distribute-by.md +++ b/docs/sql-ref-syntax-qry-select-distribute-by.md @@ -18,16 +18,21 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + +### Description + The DISTRIBUTE BY clause is used to repartition the data based on the input expressions. Unlike the [CLUSTER BY](sql-ref-syntax-qry-select-clusterby.html) clause, this does not sort the data within each partition. ### Syntax + {% highlight sql %} DISTRIBUTE BY { expression [ , ... ] } {% endhighlight %} ### Parameters +
    expression
    @@ -36,6 +41,7 @@ DISTRIBUTE BY { expression [ , ... ] }
    ### Examples + {% highlight sql %} CREATE TABLE person (name STRING, age INT); INSERT INTO person VALUES @@ -55,40 +61,39 @@ SET spark.sql.shuffle.partitions = 2; -- behavior of `DISTRIBUTE BY`. The query below produces rows where age columns are not -- clustered together. SELECT age, name FROM person; - +---+-------+ - |age|name | + |age| name| +---+-------+ - |16 |Shone S| - |25 |Zen Hui| - |16 |Jack N | - |25 |Mike A | - |18 |John A | - |18 |Anil B | + | 16|Shone S| + | 25|Zen Hui| + | 16| Jack N| + | 25| Mike A| + | 18| John A| + | 18| Anil B| +---+-------+ -- Produces rows clustered by age. Persons with same age are clustered together. -- Unlike `CLUSTER BY` clause, the rows are not sorted within a partition. SELECT age, name FROM person DISTRIBUTE BY age; - +---+-------+ - |age|name | + |age| name| +---+-------+ - |25 |Zen Hui| - |25 |Mike A | - |18 |John A | - |18 |Anil B | - |16 |Shone S| - |16 |Jack N | + | 25|Zen Hui| + | 25| Mike A| + | 18| John A| + | 18| Anil B| + | 16|Shone S| + | 16| Jack N| +---+-------+ {% endhighlight %} -### Related Clauses -- [SELECT Main](sql-ref-syntax-qry-select.html) -- [WHERE Clause](sql-ref-syntax-qry-select-where.html) -- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) -- [HAVING Clause](sql-ref-syntax-qry-select-having.html) -- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) -- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) -- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) -- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +### Related Statements + + * [SELECT Main](sql-ref-syntax-qry-select.html) + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select-groupby.md b/docs/sql-ref-syntax-qry-select-groupby.md index 49a11cafbbd5a..1676ca9c6d6aa 100644 --- a/docs/sql-ref-syntax-qry-select-groupby.md +++ b/docs/sql-ref-syntax-qry-select-groupby.md @@ -18,19 +18,30 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + +### Description + The GROUP BY clause is used to group the rows based on a set of specified grouping expressions and compute aggregations on the group of rows based on one or more specified aggregate functions. Spark also supports advanced aggregations to do multiple aggregations for the same input record set via `GROUPING SETS`, `CUBE`, `ROLLUP` clauses. +When a FILTER clause is attached to an aggregate function, only the matching rows are passed to that function. ### Syntax + {% highlight sql %} GROUP BY group_expression [ , group_expression [ , ... ] ] - [ { WITH ROLLUP | WITH CUBE | GROUPING SETS (grouping_set [ , ...]) } ] + [ { WITH ROLLUP | WITH CUBE | GROUPING SETS (grouping_set [ , ...]) } ] GROUP BY GROUPING SETS (grouping_set [ , ...]) {% endhighlight %} +While aggregate functions are defined as +{% highlight sql %} +aggregate_name ( [ DISTINCT ] expression [ , ... ] ) [ FILTER ( WHERE boolean_expression ) ] +{% endhighlight %} + ### Parameters +
    GROUPING SETS
    @@ -70,9 +81,23 @@ GROUP BY GROUPING SETS (grouping_set [ , ...]) ((warehouse, product), (warehouse), (product), ()). The N elements of a CUBE specification results in 2^N GROUPING SETS.
    +
    aggregate_name
    +
    + Specifies an aggregate function name (MIN, MAX, COUNT, SUM, AVG, etc.). +
    +
    DISTINCT
    +
    + Removes duplicates in input rows before they are passed to aggregate functions. +
    +
    FILTER
    +
    + Filters the input rows for which the boolean_expression in the WHERE clause evaluates + to true are passed to the aggregate function; other rows are discarded. +
    ### Examples + {% highlight sql %} CREATE TABLE dealer (id INT, city STRING, car_model STRING, quantity INT); INSERT INTO dealer VALUES @@ -87,39 +112,59 @@ INSERT INTO dealer VALUES -- Sum of quantity per dealership. Group by `id`. SELECT id, sum(quantity) FROM dealer GROUP BY id ORDER BY id; - +---+-------------+ - |id |sum(quantity)| + | id|sum(quantity)| +---+-------------+ - |100|32 | - |200|33 | - |300|13 | + |100| 32| + |200| 33| + |300| 13| +---+-------------+ -- Use column position in GROUP by clause. SELECT id, sum(quantity) FROM dealer GROUP BY 1 ORDER BY 1; - +---+-------------+ - |id |sum(quantity)| + | id|sum(quantity)| +---+-------------+ - |100|32 | - |200|33 | - |300|13 | + |100| 32| + |200| 33| + |300| 13| +---+-------------+ -- Multiple aggregations. -- 1. Sum of quantity per dealership. -- 2. Max quantity per dealership. SELECT id, sum(quantity) AS sum, max(quantity) AS max FROM dealer GROUP BY id ORDER BY id; - +---+---+---+ - |id |sum|max| + | id|sum|max| +---+---+---+ - |100|32 |15 | - |200|33 |20 | - |300|13 |8 | + |100| 32| 15| + |200| 33| 20| + |300| 13| 8| +---+---+---+ +-- Count the number of distinct dealer cities per car_model. +SELECT car_model, count(DISTINCT city) AS count FROM dealer GROUP BY car_model; + +------------+-----+ + | car_model|count| + +------------+-----+ + | Honda Civic| 3| + | Honda CRV| 2| + |Honda Accord| 3| + +------------+-----+ + +-- Sum of only 'Honda Civic' and 'Honda CRV' quantities per dealership. +SELECT id, sum(quantity) FILTER ( + WHERE car_model IN ('Honda Civic', 'Honda CRV') + ) AS `sum(quantity)` FROM dealer + GROUP BY id ORDER BY id; + +---+-------------+ + | id|sum(quantity)| + +---+-------------+ + |100| 17| + |200| 23| + |300| 5| + +---+-------------+ + -- Aggregations using multiple sets of grouping columns in a single statement. -- Following performs aggregations based on four sets of grouping columns. -- 1. city, car_model @@ -127,112 +172,108 @@ SELECT id, sum(quantity) AS sum, max(quantity) AS max FROM dealer GROUP BY id OR -- 3. car_model -- 4. Empty grouping set. Returns quantities for all city and car models. SELECT city, car_model, sum(quantity) AS sum FROM dealer - GROUP BY GROUPING SETS ((city, car_model), (city), (car_model), ()) - ORDER BY city; - + GROUP BY GROUPING SETS ((city, car_model), (city), (car_model), ()) + ORDER BY city; +--------+------------+---+ - |city |car_model |sum| + | city| car_model|sum| +--------+------------+---+ - |null |null |78 | - |null |Honda Accord|33 | - |null |Honda CRV |10 | - |null |Honda Civic |35 | - |Dublin |null |33 | - |Dublin |Honda Accord|10 | - |Dublin |Honda CRV |3 | - |Dublin |Honda Civic |20 | - |Fremont |null |32 | - |Fremont |Honda Accord|15 | - |Fremont |Honda CRV |7 | - |Fremont |Honda Civic |10 | - |San Jose|null |13 | - |San Jose|Honda Accord|8 | - |San Jose|Honda Civic |5 | + | null| null| 78| + | null| HondaAccord| 33| + | null| HondaCRV| 10| + | null| HondaCivic| 35| + | Dublin| null| 33| + | Dublin| HondaAccord| 10| + | Dublin| HondaCRV| 3| + | Dublin| HondaCivic| 20| + | Fremont| null| 32| + | Fremont| HondaAccord| 15| + | Fremont| HondaCRV| 7| + | Fremont| HondaCivic| 10| + | SanJose| null| 13| + | SanJose| HondaAccord| 8| + | SanJose| HondaCivic| 5| +--------+------------+---+ -- Alternate syntax for `GROUPING SETS` in which both `GROUP BY` and `GROUPING SETS` -- specifications are present. SELECT city, car_model, sum(quantity) AS sum FROM dealer - GROUP BY city, car_model GROUPING SETS ((city, car_model), (city), (car_model), ()) - ORDER BY city, car_model; - + GROUP BY city, car_model GROUPING SETS ((city, car_model), (city), (car_model), ()) + ORDER BY city, car_model; +--------+------------+---+ - |city |car_model |sum| + | city| car_model|sum| +--------+------------+---+ - |null |null |78 | - |null |Honda Accord|33 | - |null |Honda CRV |10 | - |null |Honda Civic |35 | - |Dublin |null |33 | - |Dublin |Honda Accord|10 | - |Dublin |Honda CRV |3 | - |Dublin |Honda Civic |20 | - |Fremont |null |32 | - |Fremont |Honda Accord|15 | - |Fremont |Honda CRV |7 | - |Fremont |Honda Civic |10 | - |San Jose|null |13 | - |San Jose|Honda Accord|8 | - |San Jose|Honda Civic |5 | + | null| null| 78| + | null| HondaAccord| 33| + | null| HondaCRV| 10| + | null| HondaCivic| 35| + | Dublin| null| 33| + | Dublin| HondaAccord| 10| + | Dublin| HondaCRV| 3| + | Dublin| HondaCivic| 20| + | Fremont| null| 32| + | Fremont| HondaAccord| 15| + | Fremont| HondaCRV| 7| + | Fremont| HondaCivic| 10| + | SanJose| null| 13| + | SanJose| HondaAccord| 8| + | SanJose| HondaCivic| 5| +--------+------------+---+ -- Group by processing with `ROLLUP` clause. -- Equivalent GROUP BY GROUPING SETS ((city, car_model), (city), ()) SELECT city, car_model, sum(quantity) AS sum FROM dealer - GROUP BY city, car_model WITH ROLLUP - ORDER BY city, car_model; - + GROUP BY city, car_model WITH ROLLUP + ORDER BY city, car_model; +--------+------------+---+ - |city |car_model |sum| + | city| car_model|sum| +--------+------------+---+ - |null |null |78 | - |Dublin |null |33 | - |Dublin |Honda Accord|10 | - |Dublin |Honda CRV |3 | - |Dublin |Honda Civic |20 | - |Fremont |null |32 | - |Fremont |Honda Accord|15 | - |Fremont |Honda CRV |7 | - |Fremont |Honda Civic |10 | - |San Jose|null |13 | - |San Jose|Honda Accord|8 | - |San Jose|Honda Civic |5 | + | null| null| 78| + | Dublin| null| 33| + | Dublin| HondaAccord| 10| + | Dublin| HondaCRV| 3| + | Dublin| HondaCivic| 20| + | Fremont| null| 32| + | Fremont| HondaAccord| 15| + | Fremont| HondaCRV| 7| + | Fremont| HondaCivic| 10| + | SanJose| null| 13| + | SanJose| HondaAccord| 8| + | SanJose| HondaCivic| 5| +--------+------------+---+ -- Group by processing with `CUBE` clause. -- Equivalent GROUP BY GROUPING SETS ((city, car_model), (city), (car_model), ()) SELECT city, car_model, sum(quantity) AS sum FROM dealer - GROUP BY city, car_model WITH CUBE - ORDER BY city, car_model; - + GROUP BY city, car_model WITH CUBE + ORDER BY city, car_model; +--------+------------+---+ - |city |car_model |sum| + | city| car_model|sum| +--------+------------+---+ - |null |null |78 | - |null |Honda Accord|33 | - |null |Honda CRV |10 | - |null |Honda Civic |35 | - |Dublin |null |33 | - |Dublin |Honda Accord|10 | - |Dublin |Honda CRV |3 | - |Dublin |Honda Civic |20 | - |Fremont |null |32 | - |Fremont |Honda Accord|15 | - |Fremont |Honda CRV |7 | - |Fremont |Honda Civic |10 | - |San Jose|null |13 | - |San Jose|Honda Accord|8 | - |San Jose|Honda Civic |5 | + | null| null| 78| + | null| HondaAccord| 33| + | null| HondaCRV| 10| + | null| HondaCivic| 35| + | Dublin| null| 33| + | Dublin| HondaAccord| 10| + | Dublin| HondaCRV| 3| + | Dublin| HondaCivic| 20| + | Fremont| null| 32| + | Fremont| HondaAccord| 15| + | Fremont| HondaCRV| 7| + | Fremont| HondaCivic| 10| + | SanJose| null| 13| + | SanJose| HondaAccord| 8| + | SanJose| HondaCivic| 5| +--------+------------+---+ - {% endhighlight %} -### Related clauses -- [SELECT Main](sql-ref-syntax-qry-select.html) -- [WHERE Clause](sql-ref-syntax-qry-select-where.html) -- [HAVING Clause](sql-ref-syntax-qry-select-having.html) -- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) -- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) -- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) -- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) -- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +### Related Statements + + * [SELECT Main](sql-ref-syntax-qry-select.html) + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select-having.md b/docs/sql-ref-syntax-qry-select-having.md index dee1e3c0e39b9..b84ad17955864 100644 --- a/docs/sql-ref-syntax-qry-select-having.md +++ b/docs/sql-ref-syntax-qry-select-having.md @@ -18,17 +18,22 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + +### Description + The HAVING clause is used to filter the results produced by GROUP BY based on the specified condition. It is often used in conjunction with a [GROUP BY](sql-ref-syntax-qry-select-groupby.html) clause. ### Syntax + {% highlight sql %} HAVING boolean_expression {% endhighlight %} ### Parameters +
    boolean_expression
    @@ -47,6 +52,7 @@ HAVING boolean_expression
    ### Examples + {% highlight sql %} CREATE TABLE dealer (id INT, city STRING, car_model STRING, quantity INT); INSERT INTO dealer VALUES @@ -61,16 +67,14 @@ INSERT INTO dealer VALUES -- `HAVING` clause referring to column in `GROUP BY`. SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING city = 'Fremont'; - +-------+---+ - |city |sum| + | city|sum| +-------+---+ - |Fremont|32 | + |Fremont| 32| +-------+---+ -- `HAVING` clause referring to aggregate function. SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING sum(quantity) > 15; - +-------+---+ | city|sum| +-------+---+ @@ -80,7 +84,6 @@ SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING sum(quantity) -- `HAVING` clause referring to aggregate function by its alias. SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING sum > 15; - +-------+---+ | city|sum| +-------+---+ @@ -91,16 +94,14 @@ SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING sum > 15; -- `HAVING` clause referring to a different aggregate function than what is present in -- `SELECT` list. SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING max(quantity) > 15; - +------+---+ - |city |sum| + | city|sum| +------+---+ - |Dublin|33 | + |Dublin| 33| +------+---+ -- `HAVING` clause referring to constant expression. SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING 1 > 0 ORDER BY city; - +--------+---+ | city|sum| +--------+---+ @@ -116,15 +117,15 @@ SELECT sum(quantity) AS sum FROM dealer HAVING sum(quantity) > 10; +---+ | 78| +---+ - {% endhighlight %} -### Related Clauses -- [SELECT Main](sql-ref-syntax-qry-select.html) -- [WHERE Clause](sql-ref-syntax-qry-select-where.html) -- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) -- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) -- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) -- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) -- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) -- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +### Related Statements + + * [SELECT Main](sql-ref-syntax-qry-select.html) + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select-hints.md b/docs/sql-ref-syntax-qry-select-hints.md index eb9655fc0d731..16f4f95f90ea1 100644 --- a/docs/sql-ref-syntax-qry-select-hints.md +++ b/docs/sql-ref-syntax-qry-select-hints.md @@ -1,7 +1,7 @@ --- layout: global -title: Query Hints -displayTitle: Query Hints +title: Join Hints +displayTitle: Join Hints license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -9,9 +9,9 @@ license: | 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. @@ -19,4 +19,69 @@ license: | limitations under the License. --- -**This page is under construction** +### Description + +Join Hints allow users to suggest the join strategy that Spark should use. Prior to Spark 3.0, only the `BROADCAST` Join Hint was supported. `MERGE`, `SHUFFLE_HASH` and `SHUFFLE_REPLICATE_NL` Joint Hints support was added in 3.0. When different join strategy hints are specified on both sides of a join, Spark prioritizes hints in the following order: `BROADCAST` over `MERGE` over `SHUFFLE_HASH` over `SHUFFLE_REPLICATE_NL`. When both sides are specified with the `BROADCAST` hint or the `SHUFFLE_HASH` hint, Spark will pick the build side based on the join type and the sizes of the relations. Since a given strategy may not support all join types, Spark is not guaranteed to use the join strategy suggested by the hint. + +### Join Hints Types + +
    +
    BROADCAST
    +
    + Suggests that Spark use broadcast join. The join side with the hint will be broadcast regardless of autoBroadcastJoinThreshold. If both sides of the join have the broadcast hints, the one with the smaller size (based on stats) will be broadcast. The aliases for BROADCAST are BROADCASTJOIN and MAPJOIN. +
    +
    + +
    +
    MERGE
    +
    + Suggests that Spark use shuffle sort merge join. The aliases for MERGE are SHUFFLE_MERGE and MERGEJOIN. +
    +
    + +
    +
    SHUFFLE_HASH
    +
    + Suggests that Spark use shuffle hash join. If both sides have the shuffle hash hints, Spark chooses the smaller side (based on stats) as the build side. +
    +
    + +
    +
    SHUFFLE_REPLICATE_NL
    +
    + Suggests that Spark use shuffle-and-replicate nested loop join. +
    +
    + +### Examples + +{% highlight sql %} +-- Join Hints for broadcast join +SELECT /*+ BROADCAST(t1) */ * FROM t1 INNER JOIN t2 ON t1.key = t2.key; +SELECT /*+ BROADCASTJOIN (t1) */ * FROM t1 left JOIN t2 ON t1.key = t2.key; +SELECT /*+ MAPJOIN(t2) */ * FROM t1 right JOIN t2 ON t1.key = t2.key; + +-- Join Hints for shuffle sort merge join +SELECT /*+ SHUFFLE_MERGE(t1) */ * FROM t1 INNER JOIN t2 ON t1.key = t2.key; +SELECT /*+ MERGEJOIN(t2) */ * FROM t1 INNER JOIN t2 ON t1.key = t2.key; +SELECT /*+ MERGE(t1) */ * FROM t1 INNER JOIN t2 ON t1.key = t2.key; + +-- Join Hints for shuffle hash join +SELECT /*+ SHUFFLE_HASH(t1) */ * FROM t1 INNER JOIN t2 ON t1.key = t2.key; + +-- Join Hints for shuffle-and-replicate nested loop join +SELECT /*+ SHUFFLE_REPLICATE_NL(t1) */ * FROM t1 INNER JOIN t2 ON t1.key = t2.key; + +-- When different join strategy hints are specified on both sides of a join, Spark +-- prioritizes the BROADCAST hint over the MERGE hint over the SHUFFLE_HASH hint +-- over the SHUFFLE_REPLICATE_NL hint. +-- Spark will issue Warning in the following example +-- org.apache.spark.sql.catalyst.analysis.HintErrorLogger: Hint (strategy=merge) +-- is overridden by another hint and will not take effect. +SELECT /*+ BROADCAST(t1) */ /*+ MERGE(t1, t2) */ * FROM t1 INNER JOIN t2 ON t1.key = t2.key; +{% endhighlight %} + +### Related Statements + + * [JOIN](sql-ref-syntax-qry-select-join.html) + * [SELECT](sql-ref-syntax-qry-select.html) diff --git a/docs/sql-ref-syntax-qry-select-inline-table.md b/docs/sql-ref-syntax-qry-select-inline-table.md new file mode 100644 index 0000000000000..7a532857bc322 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-inline-table.md @@ -0,0 +1,84 @@ +--- +layout: global +title: Inline Table +displayTitle: Inline Table +license: | + 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. +--- + +### Description + +An inline table is a temporary table created using a VALUES clause. + +### Syntax + +{% highlight sql %} +VALUES ( expression [ , ... ] ) [ table_alias ] +{% endhighlight %} + +### Parameters + +
    +
    expression
    +
    + Specifies a combination of one or more values, operators and SQL functions that results in a value. +
    +
    +
    +
    table_alias
    +
    + Specifies a temporary name with an optional column name list.

    + Syntax: + + [ AS ] table_name [ ( column_name [ , ... ] ) ] + +
    +
    + +### Examples + +{% highlight sql %} +-- single row, without a table alias +SELECT * FROM VALUES ("one", 1); + +----+----+ + |col1|col2| + +----+----+ + | one| 1| + +----+----+ + +-- three rows with a table alias +SELECT * FROM VALUES ("one", 1), ("two", 2), ("three", null) AS data(a, b); + +-----+----+ + | a| b| + +-----+----+ + | one| 1| + | two| 2| + |three|null| + +-----+----+ + +-- complex types with a table alias +SELECT * FROM VALUES ("one", array(0, 1)), ("two", array(2, 3)) AS data(a, b); + +---+------+ + | a| b| + +---+------+ + |one|[0, 1]| + |two|[2, 3]| + +---+------+ +{% endhighlight %} + +### Related Statement + + * [SELECT](sql-ref-syntax-qry-select.html) diff --git a/docs/sql-ref-syntax-qry-select-join.md b/docs/sql-ref-syntax-qry-select-join.md index ee66dbdb8c823..41b7603a3a25e 100644 --- a/docs/sql-ref-syntax-qry-select-join.md +++ b/docs/sql-ref-syntax-qry-select-join.md @@ -19,4 +19,244 @@ license: | limitations under the License. --- -**This page is under construction** +### Description + +A SQL join is used to combine rows from two relations based on join criteria. The following section describes the overall join syntax and the sub-sections cover different types of joins along with examples. + +### Syntax + +{% highlight sql %} +relation { [ join_type ] JOIN relation [ join_criteria ] | NATURAL join_type JOIN relation } +{% endhighlight %} + +### Parameters + +
    +
    relation
    +
    + Specifies the relation to be joined. +
    +
    join_type
    +
    + Specifies the join type.

    + Syntax:
    + + [ INNER ] + | CROSS + | LEFT [ OUTER ] + | [ LEFT ] SEMI + | RIGHT [ OUTER ] + | FULL [ OUTER ] + | [ LEFT ] ANTI + +
    +
    join_criteria
    +
    + Specifies how the rows from one relation will be combined with the rows of another relation.

    + Syntax: + + ON boolean_expression | USING ( column_name [ , column_name ... ] ) +

    + boolean_expression
    + Specifies an expression with a return type of boolean. +
    +
    + +### Join Types + +#### Inner Join + +
    +The inner join is the default join in Spark SQL. It selects rows that have matching values in both relations.

    + Syntax:
    + + relation [ INNER ] JOIN relation [ join_criteria ] + +
    + +#### Left Join + +
    +A left join returns all values from the left relation and the matched values from the right relation, or appends NULL if there is no match. It is also referred to as a left outer join.

    + Syntax:
    + + relation LEFT [ OUTER ] JOIN relation [ join_criteria ] + +
    + +#### Right Join + +
    +A right join returns all values from the right relation and the matched values from the left relation, or appends NULL if there is no match. It is also referred to as a right outer join.

    + Syntax:
    + + relation RIGHT [ OUTER ] JOIN relation [ join_criteria ] + +
    + +#### Full Join + +
    +A full join returns all values from both relations, appending NULL values on the side that does not have a match. It is also referred to as a full outer join.

    + Syntax:
    + + relation FULL [ OUTER ] JOIN relation [ join_criteria ] + +
    + +#### Cross Join + +
    +A cross join returns the Cartesian product of two relations.

    + Syntax:
    + + relation CROSS JOIN relation [ join_criteria ] + +
    + +#### Semi Join + +
    +A semi join returns values from the left side of the relation that has a match with the right. It is also referred to as a left semi join.

    + Syntax:
    + + relation [ LEFT ] SEMI JOIN relation [ join_criteria ] + +
    + +#### Anti Join + +
    +An anti join returns values from the left relation that has no match with the right. It is also referred to as a left anti join.

    + Syntax:
    + + relation [ LEFT ] ANTI JOIN relation [ join_criteria ] + +
    + +### Examples + +{% highlight sql %} +-- Use employee and department tables to demonstrate different type of joins. +SELECT * FROM employee; + + +---+-----+------+ + | id| name|deptno| + +---+-----+------+ + |105|Chloe| 5| + |103| Paul| 3| + |101| John| 1| + |102| Lisa| 2| + |104| Evan| 4| + |106| Amy| 6| + +---+-----+------+ + +SELECT * FROM department; + +------+-----------+ + |deptno| deptname| + +------+-----------+ + | 3|Engineering| + | 2| Sales| + | 1| Marketing| + +------+-----------+ + +-- Use employee and department tables to demonstrate inner join. +SELECT id, name, employee.deptno, deptname + FROM employee INNER JOIN department ON employee.deptno = department.deptno; + +---+-----+------+-----------| + | id| name|deptno| deptname| + +---+-----+------+-----------| + |103| Paul| 3|Engineering| + |101| John| 1| Marketing| + |102| Lisa| 2| Sales| + +---+-----+------+-----------| + +-- Use employee and department tables to demonstrate left join. +SELECT id, name, employee.deptno, deptname + FROM employee LEFT JOIN department ON employee.deptno = department.deptno; + +---+-----+------+-----------| + | id| name|deptno| deptname| + +---+-----+------+-----------| + |105|Chloe| 5| NULL| + |103| Paul| 3|Engineering| + |101| John| 1| Marketing| + |102| Lisa| 2| Sales| + |104| Evan| 4| NULL| + |106| Amy| 6| NULL| + +---+-----+------+-----------| + +-- Use employee and department tables to demonstrate right join. +SELECT id, name, employee.deptno, deptname + FROM employee RIGHT JOIN department ON employee.deptno = department.deptno; + +---+-----+------+-----------| + | id| name|deptno| deptname| + +---+-----+------+-----------| + |103| Paul| 3|Engineering| + |101| John| 1| Marketing| + |102| Lisa| 2| Sales| + +---+-----+------+-----------| + +-- Use employee and department tables to demonstrate full join. +SELECT id, name, employee.deptno, deptname + FROM employee FULL JOIN department ON employee.deptno = department.deptno; + +---+-----+------+-----------| + | id| name|deptno| deptname| + +---+-----+------+-----------| + |101| John| 1| Marketing| + |106| Amy| 6| NULL| + |103| Paul| 3|Engineering| + |105|Chloe| 5| NULL| + |104| Evan| 4| NULL| + |102| Lisa| 2| Sales| + +---+-----+------+-----------| + +-- Use employee and department tables to demonstrate cross join. +SELECT id, name, employee.deptno, deptname FROM employee CROSS JOIN department; + +---+-----+------+-----------| + | id| name|deptno| deptname| + +---+-----+------+-----------| + |105|Chloe| 5|Engineering| + |105|Chloe| 5| Marketing| + |105|Chloe| 5| Sales| + |103| Paul| 3|Engineering| + |103| Paul| 3| Marketing| + |103| Paul| 3| Sales| + |101| John| 1|Engineering| + |101| John| 1| Marketing| + |101| John| 1| Sales| + |102| Lisa| 2|Engineering| + |102| Lisa| 2| Marketing| + |102| Lisa| 2| Sales| + |104| Evan| 4|Engineering| + |104| Evan| 4| Marketing| + |104| Evan| 4| Sales| + |106| Amy| 4|Engineering| + |106| Amy| 4| Marketing| + |106| Amy| 4| Sales| + +---+-----+------+-----------| + +-- Use employee and department tables to demonstrate semi join. +SELECT * FROM employee SEMI JOIN department ON employee.deptno = department.deptno; + +---+-----+------+ + | id| name|deptno| + +---+-----+------+ + |103| Paul| 3| + |101| John| 1| + |102| Lisa| 2| + +---+-----+------+ + +-- Use employee and department tables to demonstrate anti join. +SELECT * FROM employee ANTI JOIN department ON employee.deptno = department.deptno; + +---+-----+------+ + | id| name|deptno| + +---+-----+------+ + |105|Chloe| 5| + |104| Evan| 4| + |106| Amy| 6| + +---+-----+------+ +{% endhighlight %} + +### Related Statements + + * [SELECT](sql-ref-syntax-qry-select.html) + * [Join Hints](sql-ref-syntax-qry-select-hints.html) diff --git a/docs/sql-ref-syntax-qry-select-limit.md b/docs/sql-ref-syntax-qry-select-limit.md index 356930c879d28..0ceb705889b47 100644 --- a/docs/sql-ref-syntax-qry-select-limit.md +++ b/docs/sql-ref-syntax-qry-select-limit.md @@ -18,17 +18,22 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + +### Description + The LIMIT clause is used to constrain the number of rows returned by the [SELECT](sql-ref-syntax-qry-select.html) statement. In general, this clause is used in conjunction with [ORDER BY](sql-ref-syntax-qry-select-orderby.html) to ensure that the results are deterministic. ### Syntax + {% highlight sql %} LIMIT { ALL | integer_expression } {% endhighlight %} ### Parameters +
    ALL
    @@ -42,6 +47,7 @@ LIMIT { ALL | integer_expression }
    ### Examples + {% highlight sql %} CREATE TABLE person (name STRING, age INT); INSERT INTO person VALUES @@ -54,31 +60,28 @@ INSERT INTO person VALUES -- Select the first two rows. SELECT name, age FROM person ORDER BY name LIMIT 2; - +------+---+ - |name |age| + | name|age| +------+---+ - |Anil B|18 | - |Jack N|16 | + |Anil B| 18| + |Jack N| 16| +------+---+ -- Specifying ALL option on LIMIT returns all the rows. SELECT name, age FROM person ORDER BY name LIMIT ALL; - +-------+---+ - |name |age| + | name|age| +-------+---+ - |Anil B |18 | - |Jack N |16 | - |John A |18 | - |Mike A |25 | - |Shone S|16 | - |Zen Hui|25 | + | Anil B| 18| + | Jack N| 16| + | John A| 18| + | Mike A| 25| + |Shone S| 16| + |Zen Hui| 25| +-------+---+ -- A function expression as an input to LIMIT. -SELECT name, age FROM person ORDER BY name LIMIT length('SPARK') - +SELECT name, age FROM person ORDER BY name LIMIT length('SPARK'); +-------+---+ | name|age| +-------+---+ @@ -90,17 +93,17 @@ SELECT name, age FROM person ORDER BY name LIMIT length('SPARK') +-------+---+ -- A non-foldable expression as an input to LIMIT is not allowed. -SELECT name, age FROM person ORDER BY name LIMIT length(name) - -org.apache.spark.sql.AnalysisException: The limit expression must evaluate to a constant value ... +SELECT name, age FROM person ORDER BY name LIMIT length(name); + org.apache.spark.sql.AnalysisException: The limit expression must evaluate to a constant value ... {% endhighlight %} -### Related Clauses -- [SELECT Main](sql-ref-syntax-qry-select.html) -- [WHERE Clause](sql-ref-syntax-qry-select-where.html) -- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) -- [HAVING Clause](sql-ref-syntax-qry-select-having.html) -- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) -- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) -- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) -- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +### Related Statements + + * [SELECT Main](sql-ref-syntax-qry-select.html) + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) diff --git a/docs/sql-ref-syntax-qry-select-orderby.md b/docs/sql-ref-syntax-qry-select-orderby.md index eb99dbb06096d..cc75367968053 100644 --- a/docs/sql-ref-syntax-qry-select-orderby.md +++ b/docs/sql-ref-syntax-qry-select-orderby.md @@ -18,16 +18,21 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + +### Description + The ORDER BY clause is used to return the result rows in a sorted manner in the user specified order. Unlike the [SORT BY](sql-ref-syntax-qry-select-sortby.html) clause, this clause guarantees a total order in the output. ### Syntax + {% highlight sql %} ORDER BY { expression [ sort_direction | nulls_sort_oder ] [ , ... ] } {% endhighlight %} ### Parameters +
    ORDER BY
    @@ -64,6 +69,7 @@ ORDER BY { expression [ sort_direction | nulls_sort_oder ] [ , ... ] }
    ### Examples + {% highlight sql %} CREATE TABLE person (id INT, name STRING, age INT); INSERT INTO person VALUES @@ -75,77 +81,73 @@ INSERT INTO person VALUES -- Sort rows by age. By default rows are sorted in ascending manner with NULL FIRST. SELECT name, age FROM person ORDER BY age; - +-----+----+ - |name |age | + | name| age| +-----+----+ |Jerry|null| - |Mary |null| - |John |30 | - |Dan |50 | - |Mike |80 | + | Mary|null| + | John| 30| + | Dan| 50| + | Mike| 80| +-----+----+ -- Sort rows in ascending manner keeping null values to be last. SELECT name, age FROM person ORDER BY age NULLS LAST; - +-----+----+ - |name |age | + | name| age| +-----+----+ - |John |30 | - |Dan |50 | - |Mike |80 | - |Mary |null| + | John| 30| + | Dan| 50| + | Mike| 80| + | Mary|null| |Jerry|null| +-----+----+ -- Sort rows by age in descending manner, which defaults to NULL LAST. SELECT name, age FROM person ORDER BY age DESC; - +-----+----+ - |name |age | + | name| age| +-----+----+ - |Mike |80 | - |Dan |50 | - |John |30 | + | Mike| 80| + | Dan| 50| + | John| 30| |Jerry|null| - |Mary |null| + | Mary|null| +-----+----+ -- Sort rows in ascending manner keeping null values to be first. SELECT name, age FROM person ORDER BY age DESC NULLS FIRST; - +-----+----+ - |name |age | + | name| age| +-----+----+ |Jerry|null| - |Mary |null| - |Mike |80 | - |Dan |50 | - |John |30 | + | Mary|null| + | Mike| 80| + | Dan| 50| + | John| 30| +-----+----+ -- Sort rows based on more than one column with each column having different -- sort direction. SELECT * FROM person ORDER BY name ASC, age DESC; - +---+-----+----+ - |id |name |age | + | id| name| age| +---+-----+----+ - |500|Dan |50 | + |500| Dan| 50| |400|Jerry|null| - |100|John |30 | - |200|Mary |null| - |300|Mike |80 | + |100| John| 30| + |200| Mary|null| + |300| Mike| 80| +---+-----+----+ {% endhighlight %} -### Related Clauses -- [SELECT Main](sql-ref-syntax-qry-select.html) -- [WHERE Clause](sql-ref-syntax-qry-select-where.html) -- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) -- [HAVING Clause](sql-ref-syntax-qry-select-having.html) -- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) -- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) -- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) -- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +### Related Statements + + * [SELECT Main](sql-ref-syntax-qry-select.html) + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select-setops.md b/docs/sql-ref-syntax-qry-select-setops.md index 526f857520aa2..09a207a70c149 100644 --- a/docs/sql-ref-syntax-qry-select-setops.md +++ b/docs/sql-ref-syntax-qry-select-setops.md @@ -1,7 +1,7 @@ --- layout: global -title: Set Operations -displayTitle: Set Operations +title: Set Operators +displayTitle: Set Operators license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,4 +19,178 @@ license: | limitations under the License. --- -**This page is under construction** +### Description + +Set operators are used to combine two input relations into a single one. Spark SQL supports three types of set operators: + + - `EXCEPT` or `MINUS` + - `INTERSECT` + - `UNION` + +Note that input relations must have the same number of columns and compatible data types for the respective columns. + +### EXCEPT + +`EXCEPT` and `EXCEPT ALL` return the rows that are found in one relation but not the other. `EXCEPT` (alternatively, `EXCEPT DISTINCT`) takes only distinct rows while `EXCEPT ALL` does not remove duplicates from the result rows. Note that `MINUS` is an alias for `EXCEPT`. + +#### Syntax + +{% highlight sql %} +[ ( ] relation [ ) ] EXCEPT | MINUS [ ALL | DISTINCT ] [ ( ] relation [ ) ] +{% endhighlight %} + +#### Examples + +{% highlight sql %} +-- Use number1 and number2 tables to demonstrate set operators in this page. +SELECT * FROM number1; + +---+ + | c| + +---+ + | 3| + | 1| + | 2| + | 2| + | 3| + | 4| + +---+ + +SELECT * FROM number2; + +---+ + | c| + +---+ + | 5| + | 1| + | 2| + | 2| + +---+ + +SELECT c FROM number1 EXCEPT SELECT c FROM number2; + +---+ + | c| + +---+ + | 3| + | 4| + +---+ + +SELECT c FROM number1 MINUS SELECT c FROM number2; + +---+ + | c| + +---+ + | 3| + | 4| + +---+ + +SELECT c FROM number1 EXCEPT ALL (SELECT c FROM number2); + +---+ + | c| + +---+ + | 3| + | 3| + | 4| + +---+ + +SELECT c FROM number1 MINUS ALL (SELECT c FROM number2); + +---+ + | c| + +---+ + | 3| + | 3| + | 4| + +---+ +{% endhighlight %} + +### INTERSECT + +`INTERSECT` and `INTERSECT ALL` return the rows that are found in both relations. `INTERSECT` (alternatively, `INTERSECT DISTINCT`) takes only distinct rows while `INTERSECT ALL` does not remove duplicates from the result rows. + +#### Syntax + +{% highlight sql %} +[ ( ] relation [ ) ] INTERSECT [ ALL | DISTINCT ] [ ( ] relation [ ) ] +{% endhighlight %} + +#### Examples + +{% highlight sql %} +(SELECT c FROM number1) INTERSECT (SELECT c FROM number2); + +---+ + | c| + +---+ + | 1| + | 2| + +---+ + +(SELECT c FROM number1) INTERSECT DISTINCT (SELECT c FROM number2); + +---+ + | c| + +---+ + | 1| + | 2| + +---+ + +(SELECT c FROM number1) INTERSECT ALL (SELECT c FROM number2); + +---+ + | c| + +---+ + | 1| + | 2| + | 2| + +---+ +{% endhighlight %} + +### UNION + +`UNION` and `UNION ALL` return the rows that are found in either relation. `UNION` (alternatively, `UNION DISTINCT`) takes only distinct rows while `UNION ALL` does not remove duplicates from the result rows. + +#### Syntax + +{% highlight sql %} +[ ( ] relation [ ) ] UNION [ ALL | DISTINCT ] [ ( ] relation [ ) ] +{% endhighlight %} + +### Examples + +{% highlight sql %} +(SELECT c FROM number1) UNION (SELECT c FROM number2); + +---+ + | c| + +---+ + | 1| + | 3| + | 5| + | 4| + | 2| + +---+ + +(SELECT c FROM number1) UNION DISTINCT (SELECT c FROM number2); + +---+ + | c| + +---+ + | 1| + | 3| + | 5| + | 4| + | 2| + +---+ + +SELECT c FROM number1 UNION ALL (SELECT c FROM number2); + +---+ + | c| + +---+ + | 3| + | 1| + | 2| + | 2| + | 3| + | 4| + | 5| + | 1| + | 2| + | 2| + +---+ +{% endhighlight %} + +### Related Statements + + * [SELECT Statement](sql-ref-syntax-qry-select.html) diff --git a/docs/sql-ref-syntax-qry-select-sortby.md b/docs/sql-ref-syntax-qry-select-sortby.md index 9b52738ee7926..315faa5e7d501 100644 --- a/docs/sql-ref-syntax-qry-select-sortby.md +++ b/docs/sql-ref-syntax-qry-select-sortby.md @@ -18,6 +18,9 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + +### Description + The SORT BY clause is used to return the result rows sorted within each partition in the user specified order. When there is more than one partition SORT BY may return result that is partially ordered. This is different @@ -25,11 +28,13 @@ than [ORDER BY](sql-ref-syntax-qry-select-orderby.html) clause which guarantees total order of the output. ### Syntax + {% highlight sql %} SORT BY { expression [ sort_direction | nulls_sort_order ] [ , ... ] } {% endhighlight %} ### Parameters +
    SORT BY
    @@ -66,6 +71,7 @@ SORT BY { expression [ sort_direction | nulls_sort_order ] [ , ... ] }
    ### Examples + {% highlight sql %} CREATE TABLE person (zip_code INT, name STRING, age INT); INSERT INTO person VALUES @@ -83,103 +89,98 @@ INSERT INTO person VALUES -- Sort rows by `name` within each partition in ascending manner SELECT /*+ REPARTITION(zip_code) */ name, age, zip_code FROM person SORT BY name; - +--------+----+--------+ - |name |age |zip_code| + | name| age|zip_code| +--------+----+--------+ - |Anil K |27 |94588 | - |Dan Li |18 |94588 | - |John V |null|94588 | - |Zen Hui |50 |94588 | - |Aryan B.|18 |94511 | - |David K |42 |94511 | - |Lalit B.|null|94511 | + | Anil K| 27| 94588| + | Dan Li| 18| 94588| + | John V|null| 94588| + | Zen Hui| 50| 94588| + |Aryan B.| 18| 94511| + | David K| 42| 94511| + |Lalit B.|null| 94511| +--------+----+--------+ -- Sort rows within each partition using column position. SELECT /*+ REPARTITION(zip_code) */ name, age, zip_code FROM person SORT BY 1; - +--------+----+--------+ - |name |age |zip_code| + | name| age|zip_code| +--------+----+--------+ - |Anil K |27 |94588 | - |Dan Li |18 |94588 | - |John V |null|94588 | - |Zen Hui |50 |94588 | - |Aryan B.|18 |94511 | - |David K |42 |94511 | - |Lalit B.|null|94511 | + | Anil K| 27| 94588| + | Dan Li| 18| 94588| + | John V|null| 94588| + | Zen Hui| 50| 94588| + |Aryan B.| 18| 94511| + | David K| 42| 94511| + |Lalit B.|null| 94511| +--------+----+--------+ -- Sort rows within partition in ascending manner keeping null values to be last. SELECT /*+ REPARTITION(zip_code) */ age, name, zip_code FROM person SORT BY age NULLS LAST; - +----+--------+--------+ - |age |name |zip_code| + | age| name|zip_code| +----+--------+--------+ - |18 |Dan Li |94588 | - |27 |Anil K |94588 | - |50 |Zen Hui |94588 | - |null|John V |94588 | - |18 |Aryan B.|94511 | - |42 |David K |94511 | - |null|Lalit B.|94511 | + | 18| Dan Li| 94588| + | 27| Anil K| 94588| + | 50| Zen Hui| 94588| + |null| John V| 94588| + | 18|Aryan B.| 94511| + | 42| David K| 94511| + |null|Lalit B.| 94511| +----+--------+--------+ -- Sort rows by age within each partition in descending manner, which defaults to NULL LAST. SELECT /*+ REPARTITION(zip_code) */ age, name, zip_code FROM person SORT BY age DESC; - +----+--------+--------+ - |age |name |zip_code| + | age| name|zip_code| +----+--------+--------+ - |50 |Zen Hui |94588 | - |27 |Anil K |94588 | - |18 |Dan Li |94588 | - |null|John V |94588 | - |42 |David K |94511 | - |18 |Aryan B.|94511 | - |null|Lalit B.|94511 | + | 50| Zen Hui| 94588| + | 27| Anil K| 94588| + | 18| Dan Li| 94588| + |null| John V| 94588| + | 42| David K| 94511| + | 18|Aryan B.| 94511| + |null|Lalit B.| 94511| +----+--------+--------+ -- Sort rows by age within each partition in descending manner keeping null values to be first. SELECT /*+ REPARTITION(zip_code) */ age, name, zip_code FROM person SORT BY age DESC NULLS FIRST; - +----+--------+--------+ - |age |name |zip_code| + | age| name|zip_code| +----+--------+--------+ - |null|John V |94588 | - |50 |Zen Hui |94588 | - |27 |Anil K |94588 | - |18 |Dan Li |94588 | - |null|Lalit B.|94511 | - |42 |David K |94511 | - |18 |Aryan B.|94511 | + |null| John V| 94588| + | 50| Zen Hui| 94588| + | 27| Anil K| 94588| + | 18| Dan Li| 94588| + |null|Lalit B.| 94511| + | 42| David K| 94511| + | 18|Aryan B.| 94511| +----+--------+--------+ -- Sort rows within each partition based on more than one column with each column having -- different sort direction. SELECT /*+ REPARTITION(zip_code) */ name, age, zip_code FROM person - SORT BY name ASC, age DESC; - + SORT BY name ASC, age DESC; +--------+----+--------+ - |name |age |zip_code| + | name| age|zip_code| +--------+----+--------+ - |Anil K |27 |94588 | - |Dan Li |18 |94588 | - |John V |null|94588 | - |Zen Hui |50 |94588 | - |Aryan B.|18 |94511 | - |David K |42 |94511 | - |Lalit B.|null|94511 | + | Anil K| 27| 94588| + | Dan Li| 18| 94588| + | John V|null| 94588| + | Zen Hui| 50| 94588| + |Aryan B.| 18| 94511| + | David K| 42| 94511| + |Lalit B.|null| 94511| +--------+----+--------+ {% endhighlight %} -### Related Clauses -- [SELECT Main](sql-ref-syntax-qry-select.html) -- [WHERE Clause](sql-ref-syntax-qry-select-where.html) -- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) -- [HAVING Clause](sql-ref-syntax-qry-select-having.html) -- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) -- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) -- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) -- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +### Related Statements + + * [SELECT Main](sql-ref-syntax-qry-select.html) + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select-tvf.md b/docs/sql-ref-syntax-qry-select-tvf.md new file mode 100644 index 0000000000000..bbfd870d0cbe7 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-tvf.md @@ -0,0 +1,131 @@ +--- +layout: global +title: Table-valued Functions (TVF) +displayTitle: Table-valued Functions (TVF) +license: | + 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. +--- + +### Description + +A table-valued function (TVF) is a function that returns a relation or a set of rows. + +### Syntax + +{% highlight sql %} +function_name ( expression [ , ... ] ) [ table_alias ] +{% endhighlight %} + +### Parameters + +
    +
    expression
    +
    + Specifies a combination of one or more values, operators and SQL functions that results in a value. +
    +
    +
    +
    table_alias
    +
    + Specifies a temporary name with an optional column name list.

    + Syntax: + + [ AS ] table_name [ ( column_name [ , ... ] ) ] + +
    +
    + +### Supported Table-valued Functions + +
    + + + + + + + + + + + + + + + + + + + + + + + +
    FunctionArgument Type(s)Description
    range ( end ) Long Creates a table with a single LongType column named id, containing rows in a range from 0 to end (exclusive) with step value 1.
    range ( start, end ) Long, Long Creates a table with a single LongType column named id, containing rows in a range from start to end (exclusive) with step value 1.
    range ( start, end, step ) Long, Long, Long Creates a table with a single LongType column named id, containing rows in a range from start to end (exclusive) with step value.
    range ( start, end, step, numPartitions ) Long, Long, Long, Int Creates a table with a single LongType column named id, containing rows in a range from start to end (exclusive) with step value, with partition number numPartitions specified.
    + +### Examples + +{% highlight sql %} +-- range call with end +SELECT * FROM range(6 + cos(3)); + +---+ + | id| + +---+ + | 0| + | 1| + | 2| + | 3| + | 4| + +---+ + +-- range call with start and end +SELECT * FROM range(5, 10); + +---+ + | id| + +---+ + | 5| + | 6| + | 7| + | 8| + | 9| + +---+ + +-- range call with numPartitions +SELECT * FROM range(0, 10, 2, 200); + +---+ + | id| + +---+ + | 0| + | 2| + | 4| + | 6| + | 8| + +---+ + +-- range call with a table alias +SELECT * FROM range(5, 8) AS test; + +---+ + | id| + +---+ + | 5| + | 6| + | 7| + +---+ +{% endhighlight %} + +### Related Statement + + * [SELECT](sql-ref-syntax-qry-select.html) diff --git a/docs/sql-ref-syntax-qry-select-usedb.md b/docs/sql-ref-syntax-qry-select-usedb.md index 92ac91ac51769..2a05085218978 100644 --- a/docs/sql-ref-syntax-qry-select-usedb.md +++ b/docs/sql-ref-syntax-qry-select-usedb.md @@ -20,12 +20,14 @@ license: | --- ### Description + `USE` statement is used to set the current database. After the current database is set, the unqualified database artifacts such as tables, functions and views that are referenced by SQLs are resolved from the current database. The default database name is 'default'. ### Syntax + {% highlight sql %} USE database_name {% endhighlight %} @@ -40,21 +42,18 @@ USE database_name ### Example + {% highlight sql %} -- Use the 'userdb' which exists. USE userdb; -+---------+--+ -| Result | -+---------+--+ -+---------+--+ -- Use the 'userdb1' which doesn't exist USE userdb1; -Error: org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database 'userdb1' not found;(state=,code=0) + Error: org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database 'userdb1' not found;(state=,code=0) {% endhighlight %} -### Related statements. -- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) -- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) -- [CREATE TABLE ](sql-ref-syntax-ddl-create-table.html) +### Related Statements + * [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) + * [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) + * [CREATE TABLE ](sql-ref-syntax-ddl-create-table.html) diff --git a/docs/sql-ref-syntax-qry-select-where.md b/docs/sql-ref-syntax-qry-select-where.md index 106053d16f8bd..1960367cd42f0 100644 --- a/docs/sql-ref-syntax-qry-select-where.md +++ b/docs/sql-ref-syntax-qry-select-where.md @@ -18,15 +18,20 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + +### Description + The WHERE clause is used to limit the results of the FROM clause of a query or a subquery based on the specified condition. ### Syntax + {% highlight sql %} WHERE boolean_expression {% endhighlight %} ### Parameters +
    boolean_expression
    @@ -37,6 +42,7 @@ WHERE boolean_expression
    ### Examples + {% highlight sql %} CREATE TABLE person (id INT, name STRING, age INT); INSERT INTO person VALUES @@ -48,38 +54,38 @@ INSERT INTO person VALUES -- Comparison operator in `WHERE` clause. SELECT * FROM person WHERE id > 200 ORDER BY id; +---+----+---+ - |id |name|age| + | id|name|age| +---+----+---+ - |300|Mike|80 | - |400|Dan |50 | + |300|Mike| 80| + |400| Dan| 50| +---+----+---+ -- Comparison and logical operators in `WHERE` clause. SELECT * FROM person WHERE id = 200 OR id = 300 ORDER BY id; +---+----+----+ - |id |name|age | + | id|name| age| +---+----+----+ |200|Mary|null| - |300|Mike|80 | + |300|Mike| 80| +---+----+----+ -- IS NULL expression in `WHERE` clause. SELECT * FROM person WHERE id > 300 OR age IS NULL ORDER BY id; +---+----+----+ - |id |name|age | + | id|name| age| +---+----+----+ |200|Mary|null| - |400|Dan |50 | + |400| Dan| 50| +---+----+----+ -- Function expression in `WHERE` clause. SELECT * FROM person WHERE length(name) > 3 ORDER BY id; +---+----+----+ - |id |name|age | + | id|name| age| +---+----+----+ - |100|John|30 | + |100|John| 30| |200|Mary|null| - |300|Mike|80 | + |300|Mike| 80| +---+----+----+ -- `BETWEEN` expression in `WHERE` clause. @@ -94,31 +100,31 @@ SELECT * FROM person WHERE id BETWEEN 200 AND 300 ORDER BY id; -- Scalar Subquery in `WHERE` clause. SELECT * FROM person WHERE age > (SELECT avg(age) FROM person); +---+----+---+ - |id |name|age| + | id|name|age| +---+----+---+ - |300|Mike|80 | + |300|Mike| 80| +---+----+---+ -- Correlated Subquery in `WHERE` clause. SELECT * FROM person AS parent -WHERE EXISTS ( - SELECT 1 FROM person AS child - WHERE parent.id = child.id AND child.age IS NULL - ); + WHERE EXISTS ( + SELECT 1 FROM person AS child + WHERE parent.id = child.id AND child.age IS NULL + ); +---+----+----+ |id |name|age | +---+----+----+ |200|Mary|null| +---+----+----+ - {% endhighlight %} -### Related Clauses -- [SELECT Main](sql-ref-syntax-qry-select.html) -- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) -- [HAVING Clause](sql-ref-syntax-qry-select-having.html) -- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) -- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) -- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) -- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) -- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +### Related Statements + + * [SELECT Main](sql-ref-syntax-qry-select.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select.md b/docs/sql-ref-syntax-qry-select.md index e87c4a587e34c..94f69d4d733c4 100644 --- a/docs/sql-ref-syntax-qry-select.md +++ b/docs/sql-ref-syntax-qry-select.md @@ -18,33 +18,38 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- + +### Description + Spark supports a `SELECT` statement and conforms to the ANSI SQL standard. Queries are used to retrieve result sets from one or more tables. The following section describes the overall query syntax and the sub-sections cover different constructs of a query along with examples. ### Syntax + {% highlight sql %} [ WITH with_query [ , ... ] ] select_statement [ { UNION | INTERSECT | EXCEPT } [ ALL | DISTINCT ] select_statement, ... ] -[ ORDER BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] -[ SORT BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] -[ CLUSTER BY { expression [ , ...] } ] -[ DISTRIBUTE BY { expression [, ...] } ] -[ WINDOW { named_window [ , WINDOW named_window, ... ] } ] -[ LIMIT { ALL | expression } ] + [ ORDER BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] + [ SORT BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] + [ CLUSTER BY { expression [ , ...] } ] + [ DISTRIBUTE BY { expression [, ...] } ] + [ WINDOW { named_window [ , WINDOW named_window, ... ] } ] + [ LIMIT { ALL | expression } ] {% endhighlight %} While `select_statement` is defined as {% highlight sql %} SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } - FROM { from_item [ , ...] } - [ WHERE boolean_expression ] - [ GROUP BY expression [ , ...] ] - [ HAVING boolean_expression ] + FROM { from_item [ , ...] } + [ WHERE boolean_expression ] + [ GROUP BY expression [ , ...] ] + [ HAVING boolean_expression ] {% endhighlight %} ### Parameters +
    with_query
    @@ -78,9 +83,9 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } Specifies a source of input for the query. It can be one of the following:
    1. Table relation
    2. -
    3. Join relation
    4. -
    5. Table valued function
    6. -
    7. Inlined table
    8. +
    9. Join relation
    10. +
    11. Table-value function
    12. +
    13. Inline table
    14. Subquery
    @@ -92,6 +97,7 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] }
    Specifies the expressions that are used to group the rows. This is used in conjunction with aggregate functions (MIN, MAX, COUNT, SUM, AVG, etc.) to group rows based on the grouping expressions and aggregate values in each group. + When a FILTER clause is attached to an aggregate function, only the matching rows are passed to that function.
    HAVING
    @@ -140,12 +146,16 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] }
    -### Related Clauses -- [WHERE Clause](sql-ref-syntax-qry-select-where.html) -- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) -- [HAVING Clause](sql-ref-syntax-qry-select-having.html) -- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) -- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) -- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) -- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) -- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +### Related Statements + + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) + * [TABLESAMPLE](sql-ref-syntax-qry-sampling.html) + * [JOIN](sql-ref-syntax-qry-select-join.html) + * [SET Operators](sql-ref-syntax-qry-select-setops.html) diff --git a/docs/sql-ref-syntax-qry.md b/docs/sql-ref-syntax-qry.md index 37414acd57a38..477c347eed800 100644 --- a/docs/sql-ref-syntax-qry.md +++ b/docs/sql-ref-syntax-qry.md @@ -26,13 +26,12 @@ and brief description of supported clauses are explained in ability to generate logical and physical plan for a given query using [EXPLAIN](sql-ref-syntax-qry-explain.html) statement. - -- [WHERE Clause](sql-ref-syntax-qry-select-where.html) -- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) -- [HAVING Clause](sql-ref-syntax-qry-select-having.html) -- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) -- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) -- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) -- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) -- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) -- [EXPLAIN Statement](sql-ref-syntax-qry-explain.html) + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) + * [EXPLAIN Statement](sql-ref-syntax-qry-explain.html) diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md index 2510278ab252c..94bd476ffb7b1 100644 --- a/docs/sql-ref-syntax.md +++ b/docs/sql-ref-syntax.md @@ -19,4 +19,69 @@ license: | limitations under the License. --- -Spark SQL is Apache Spark's module for working with structured data. The SQL Syntax section describes the SQL syntax in detail along with usage examples when applicable. +Spark SQL is Apache Spark's module for working with structured data. The SQL Syntax section describes the SQL syntax in detail along with usage examples when applicable. This document provides a list of Data Definition and Data Manipulation Statements, as well as Data Retrieval and Auxiliary Statements. + +### DDL Statements + + * [ALTER DATABASE](sql-ref-syntax-ddl-alter-database.html) + * [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) + * [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) + * [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) + * [CREATE FUNCTION](sql-ref-syntax-ddl-create-function.html) + * [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) + * [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) + * [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) + * [DROP FUNCTION](sql-ref-syntax-ddl-drop-function.html) + * [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) + * [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) + * [REPAIR TABLE](sql-ref-syntax-ddl-repair-table.html) + * [TRUNCATE TABLE](sql-ref-syntax-ddl-truncate-table.html) + * [USE DATABASE](sql-ref-syntax-qry-select-usedb.html) + +### DML Statements + + * [INSERT INTO](sql-ref-syntax-dml-insert-into.html) + * [INSERT OVERWRITE](sql-ref-syntax-dml-insert-overwrite-table.html) + * [INSERT OVERWRITE DIRECTORY](sql-ref-syntax-dml-insert-overwrite-directory.html) + * [INSERT OVERWRITE DIRECTORY with Hive format](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) + * [LOAD](sql-ref-syntax-dml-load.html) + +### Data Retrieval Statements + + * [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) + * [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) + * [EXPLAIN](sql-ref-syntax-qry-explain.html) + * [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) + * [HAVING Clause](sql-ref-syntax-qry-select-having.html) + * [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) + * [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) + * [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) + * [WHERE Clause](sql-ref-syntax-qry-select-where.html) + +### Auxiliary Statements + + * [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) + * [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) + * [ANALYZE TABLE](sql-ref-syntax-aux-analyze-table.html) + * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) + * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) + * [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) + * [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) + * [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) + * [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) + * [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) + * [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) + * [REFRESH](sql-ref-syntax-aux-cache-refresh.html) + * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) + * [SET](sql-ref-syntax-aux-conf-mgmt-set.html) + * [SHOW COLUMNS](sql-ref-syntax-aux-show-columns.html) + * [SHOW CREATE TABLE](sql-ref-syntax-aux-show-create-table.html) + * [SHOW DATABASES](sql-ref-syntax-aux-show-databases.html) + * [SHOW FUNCTIONS](sql-ref-syntax-aux-show-functions.html) + * [SHOW PARTITIONS](sql-ref-syntax-aux-show-partitions.html) + * [SHOW TABLE EXTENDED](sql-ref-syntax-aux-show-table.html) + * [SHOW TABLES](sql-ref-syntax-aux-show-tables.html) + * [SHOW TBLPROPERTIES](sql-ref-syntax-aux-show-tblproperties.html) + * [SHOW VIEWS](sql-ref-syntax-aux-show-views.html) + * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) + * [UNSET](sql-ref-syntax-aux-conf-mgmt-reset.html) diff --git a/docs/ss-migration-guide.md b/docs/ss-migration-guide.md index db8fdff8b2ac4..963ef07af7ace 100644 --- a/docs/ss-migration-guide.md +++ b/docs/ss-migration-guide.md @@ -30,4 +30,4 @@ Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide. - In Spark 3.0, Structured Streaming forces the source schema into nullable when file-based datasources such as text, json, csv, parquet and orc are used via `spark.readStream(...)`. Previously, it respected the nullability in source schema; however, it caused issues tricky to debug with NPE. To restore the previous behavior, set `spark.sql.streaming.fileSource.schema.forceNullable` to `false`. -- Spark 3.0 fixes the correctness issue on Stream-stream outer join, which changes the schema of state. (SPARK-26154 for more details) Spark 3.0 will fail the query if you start your query from checkpoint constructed from Spark 2.x which uses stream-stream outer join. Please discard the checkpoint and replay previous inputs to recalculate outputs. \ No newline at end of file +- Spark 3.0 fixes the correctness issue on Stream-stream outer join, which changes the schema of state. (See [SPARK-26154](https://issues.apache.org/jira/browse/SPARK-26154) for more details). If you start your query from checkpoint constructed from Spark 2.x which uses stream-stream outer join, Spark 3.0 fails the query. To recalculate outputs, discard the checkpoint and replay previous inputs. diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index e68d51321cae3..db813c46949c2 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -246,8 +246,7 @@ To run the example,
    - ./bin/spark-submit --jars external/kinesis-asl/target/scala-*/\ - spark-streaming-kinesis-asl-assembly_*.jar \ + ./bin/spark-submit --jars 'external/kinesis-asl-assembly/target/spark-streaming-kinesis-asl-assembly_*.jar' \ external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py \ [Kinesis app name] [Kinesis stream name] [endpoint URL] [region name] diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index a1eeee54987db..016faa735acd6 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -525,21 +525,24 @@ The caching key is built up from the following information: The following properties are available to configure the consumer pool: - + + + + @@ -547,6 +550,7 @@ The following properties are available to configure the consumer pool: The prefix of JMX name is set to "kafka010-cached-simple-kafka-consumer-pool". +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.kafka.consumer.cache.capacity The maximum number of consumers cached. Please note that it's a soft limit. 643.0.0
    spark.kafka.consumer.cache.timeout The minimum amount of time a consumer may sit idle in the pool before it is eligible for eviction by the evictor. 5m (5 minutes)3.0.0
    spark.kafka.consumer.cache.evictorThreadRunInterval The interval of time between runs of the idle evictor thread for consumer pool. When non-positive, no idle evictor thread will be run. 1m (1 minute)3.0.0
    spark.kafka.consumer.cache.jmx.enable false3.0.0
    @@ -571,16 +575,18 @@ Note that it doesn't leverage Apache Commons Pool due to the difference of chara The following properties are available to configure the fetched data pool: - + + +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.kafka.consumer.fetchedData.cache.timeout The minimum amount of time a fetched data may sit idle in the pool before it is eligible for eviction by the evictor. 5m (5 minutes)3.0.0
    spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval The interval of time between runs of the idle evictor thread for fetched data pool. When non-positive, no idle evictor thread will be run. 1m (1 minute)3.0.0
    @@ -816,16 +822,18 @@ It will use different Kafka producer when delegation token is renewed; Kafka pro The following properties are available to configure the producer pool: - + + +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.kafka.producer.cache.timeout The minimum amount of time a producer may sit idle in the pool before it is eligible for eviction by the evictor. 10m (10 minutes)2.2.1
    spark.kafka.producer.cache.evictorThreadRunInterval The interval of time between runs of the idle evictor thread for producer pool. When non-positive, no idle evictor thread will be run. 1m (1 minute)3.0.0
    @@ -935,7 +943,7 @@ When none of the above applies then unsecure connection assumed. Delegation tokens can be obtained from multiple clusters and ${cluster} is an arbitrary unique identifier which helps to group different configurations. - + @@ -943,6 +951,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster} + @@ -953,6 +962,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster} + @@ -962,6 +972,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster}bootstrap.servers config matches (for further details please see spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex), and can be overridden by setting kafka.security.protocol on the source or sink. + @@ -970,6 +981,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster} + @@ -977,6 +989,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster} The location of the trust store file. For further details please see Kafka documentation. Only used to obtain delegation token. + @@ -985,6 +998,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster}spark.kafka.clusters.${cluster}.ssl.truststore.location is configured. For further details please see Kafka documentation. Only used to obtain delegation token. + @@ -993,6 +1007,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster} + @@ -1001,6 +1016,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster}spark.kafka.clusters.${cluster}.ssl.keystore.location is configured. For further details please see Kafka documentation. Only used to obtain delegation token. + @@ -1009,6 +1025,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster} + @@ -1017,6 +1034,7 @@ Delegation tokens can be obtained from multiple clusters and ${cluster}sasl.mechanism). Only used to authenticate against Kafka broker with delegation token. +
    Property NameDefaultMeaning
    Property NameDefaultMeaningSince Version
    spark.kafka.clusters.${cluster}.auth.bootstrap.servers None3.0.0
    spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex3.0.0
    spark.kafka.clusters.${cluster}.security.protocol3.0.0
    spark.kafka.clusters.${cluster}.sasl.kerberos.service.name3.0.0
    spark.kafka.clusters.${cluster}.ssl.truststore.location3.0.0
    spark.kafka.clusters.${cluster}.ssl.truststore.password3.0.0
    spark.kafka.clusters.${cluster}.ssl.keystore.location3.0.0
    spark.kafka.clusters.${cluster}.ssl.keystore.password3.0.0
    spark.kafka.clusters.${cluster}.ssl.key.password3.0.0
    spark.kafka.clusters.${cluster}.sasl.token.mechanism3.0.0
    diff --git a/docs/web-ui.md b/docs/web-ui.md index e28a689c8de50..c53af804d8d59 100644 --- a/docs/web-ui.md +++ b/docs/web-ui.md @@ -143,6 +143,7 @@ Summary metrics for all task are represented in a table and in a timeline. * **Shuffle Read Size / Records**. Total shuffle bytes read, includes both data read locally and data read from remote executors. * **Shuffle Read Blocked Time** is the time that tasks spent blocked waiting for shuffle data to be read from remote machines. * **Shuffle Remote Reads** is the total shuffle bytes read from remote executors. +* **Shuffle Write Time** is the time that tasks spent writing shuffle data. * **Shuffle spill (memory)** is the size of the deserialized form of the shuffled data in memory. * **Shuffle spill (disk)** is the size of the serialized form of the data on disk. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaANOVATestExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaANOVATestExample.java new file mode 100644 index 0000000000000..3b2de1f39cc88 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaANOVATestExample.java @@ -0,0 +1,75 @@ +/* + * 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.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.stat.ANOVATest; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.*; +// $example off$ + +/** + * An example for ANOVA testing. + * Run with + *
    + * bin/run-example ml.JavaANOVATestExample
    + * 
    + */ +public class JavaANOVATestExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaANOVATestExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(3.0, Vectors.dense(1.7, 4.4, 7.6, 5.8, 9.6, 2.3)), + RowFactory.create(2.0, Vectors.dense(8.8, 7.3, 5.7, 7.3, 2.2, 4.1)), + RowFactory.create(1.0, Vectors.dense(1.2, 9.5, 2.5, 3.1, 8.7, 2.5)), + RowFactory.create(2.0, Vectors.dense(3.7, 9.2, 6.1, 4.1, 7.5, 3.8)), + RowFactory.create(4.0, Vectors.dense(8.9, 5.2, 7.8, 8.3, 5.2, 3.0)), + RowFactory.create(4.0, Vectors.dense(7.9, 8.5, 9.2, 4.0, 9.4, 2.1)) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()), + }); + + Dataset df = spark.createDataFrame(data, schema); + Row r = ANOVATest.test(df, "features", "label").head(); + System.out.println("pValues: " + r.get(0).toString()); + System.out.println("degreesOfFreedom: " + r.getList(1).toString()); + System.out.println("fValues: " + r.get(2).toString()); + + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedScalar.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedScalar.java new file mode 100644 index 0000000000000..e5e698809b215 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedScalar.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql; + +// $example on:udf_scalar$ +import org.apache.spark.sql.*; +import org.apache.spark.sql.api.java.UDF1; +import org.apache.spark.sql.expressions.UserDefinedFunction; +import static org.apache.spark.sql.functions.udf; +import org.apache.spark.sql.types.DataTypes; +// $example off:udf_scalar$ + +public class JavaUserDefinedScalar { + + public static void main(String[] args) { + + // $example on:udf_scalar$ + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL UDF scalar example") + .getOrCreate(); + + // Define and register a zero-argument non-deterministic UDF + // UDF is deterministic by default, i.e. produces the same result for the same input. + UserDefinedFunction random = udf( + () -> Math.random(), DataTypes.DoubleType + ); + random.asNondeterministic(); + spark.udf().register("random", random); + spark.sql("SELECT random()").show(); + // +-------+ + // |UDF() | + // +-------+ + // |xxxxxxx| + // +-------+ + + // Define and register a one-argument UDF + spark.udf().register("plusOne", new UDF1() { + @Override + public Integer call(Integer x) { + return x + 1; + } + }, DataTypes.IntegerType); + spark.sql("SELECT plusOne(5)").show(); + // +----------+ + // |plusOne(5)| + // +----------+ + // | 6| + // +----------+ + + // Define and register a two-argument UDF + UserDefinedFunction strLen = udf( + (String s, Integer x) -> s.length() + x, DataTypes.IntegerType + ); + spark.udf().register("strLen", strLen); + spark.sql("SELECT strLen('test', 1)").show(); + // +------------+ + // |UDF(test, 1)| + // +------------+ + // | 5| + // +------------+ + + // UDF in a WHERE clause + spark.udf().register("oneArgFilter", new UDF1() { + @Override + public Boolean call(Long x) { + return x > 5; + } + }, DataTypes.BooleanType); + spark.range(1, 10).createOrReplaceTempView("test"); + spark.sql("SELECT * FROM test WHERE oneArgFilter(id)").show(); + // +---+ + // | id| + // +---+ + // | 6| + // | 7| + // | 8| + // | 9| + // +---+ + + // $example off:udf_scalar$ + spark.stop(); + } +} diff --git a/examples/src/main/python/ml/anova_test_example.py b/examples/src/main/python/ml/anova_test_example.py new file mode 100644 index 0000000000000..3fffdbddf3aca --- /dev/null +++ b/examples/src/main/python/ml/anova_test_example.py @@ -0,0 +1,52 @@ +# +# 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. +# + +""" +An example for ANOVA testing. +Run with: + bin/spark-submit examples/src/main/python/ml/anova_test_example.py +""" +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.linalg import Vectors +from pyspark.ml.stat import ANOVATest +# $example off$ + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("ANOVATestExample")\ + .getOrCreate() + + # $example on$ + data = [(3.0, Vectors.dense([1.7, 4.4, 7.6, 5.8, 9.6, 2.3])), + (2.0, Vectors.dense([8.8, 7.3, 5.7, 7.3, 2.2, 4.1])), + (1.0, Vectors.dense([1.2, 9.5, 2.5, 3.1, 8.7, 2.5])), + (2.0, Vectors.dense([3.7, 9.2, 6.1, 4.1, 7.5, 3.8])), + (4.0, Vectors.dense([8.9, 5.2, 7.8, 8.3, 5.2, 3.0])), + (4.0, Vectors.dense([7.9, 8.5, 9.2, 4.0, 9.4, 2.1]))] + df = spark.createDataFrame(data, ["label", "features"]) + + r = ANOVATest.test(df, "features", "label").head() + print("pValues: " + str(r.pValues)) + print("degreesOfFreedom: " + str(r.degreesOfFreedom)) + print("fValues: " + str(r.fValues)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/r/ml/fmClassifier.R b/examples/src/main/r/ml/fmClassifier.R new file mode 100644 index 0000000000000..3f9df91a884a6 --- /dev/null +++ b/examples/src/main/r/ml/fmClassifier.R @@ -0,0 +1,45 @@ +# +# 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. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/fmClassifier.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-fmclasfier-example") + +# $example on$ +# Load training data +df <- read.df("data/mllib/sample_libsvm_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a FM classification model +model <- spark.fmClassifier(training, label ~ features) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) + +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/fmRegressor.R b/examples/src/main/r/ml/fmRegressor.R new file mode 100644 index 0000000000000..7baa1b5398e0a --- /dev/null +++ b/examples/src/main/r/ml/fmRegressor.R @@ -0,0 +1,45 @@ +# +# 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. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/fmRegressor.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-fmRegressor-example") + +# $example on$ +# Load training data +df <- read.df("data/mllib/sample_linear_regression_data.txt", source = "libsvm") +training_test <- randomSplit(df, c(0.7, 0.3)) +training <- training_test[[1]] +test <- training_test[[2]] + +# Fit a FM regression model +model <- spark.fmRegressor(training, label ~ features) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/lm_with_elastic_net.R b/examples/src/main/r/ml/lm_with_elastic_net.R new file mode 100644 index 0000000000000..5294ca174bf18 --- /dev/null +++ b/examples/src/main/r/ml/lm_with_elastic_net.R @@ -0,0 +1,45 @@ +# +# 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. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/lm_with_elastic_net.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-lm-example") + +# $example on$ +# Load training data +df <- read.df("data/mllib/sample_linear_regression_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a linear regression model +model <- spark.lm(training, label ~ features, regParam = 0.3, elasticNetParam = 0.8) + +# Prediction +predictions <- predict(model, test) +head(predictions) + +# Summarize +summary(model) + +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ANOVATestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ANOVATestExample.scala new file mode 100644 index 0000000000000..0cd793f5b7b88 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ANOVATestExample.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.stat.ANOVATest +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example for ANOVA testing. + * Run with + * {{{ + * bin/run-example ml.ANOVATestExample + * }}} + */ +object ANOVATestExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("ANOVATestExample") + .getOrCreate() + import spark.implicits._ + + // $example on$ + val data = Seq( + (3.0, Vectors.dense(1.7, 4.4, 7.6, 5.8, 9.6, 2.3)), + (2.0, Vectors.dense(8.8, 7.3, 5.7, 7.3, 2.2, 4.1)), + (1.0, Vectors.dense(1.2, 9.5, 2.5, 3.1, 8.7, 2.5)), + (2.0, Vectors.dense(3.7, 9.2, 6.1, 4.1, 7.5, 3.8)), + (4.0, Vectors.dense(8.9, 5.2, 7.8, 8.3, 5.2, 3.0)), + (4.0, Vectors.dense(7.9, 8.5, 9.2, 4.0, 9.4, 2.1)) + ) + + val df = data.toDF("label", "features") + val anova = ANOVATest.test(df, "features", "label").head + println(s"pValues = ${anova.getAs[Vector](0)}") + println(s"degreesOfFreedom ${anova.getSeq[Int](1).mkString("[", ",", "]")}") + println(s"fValues ${anova.getAs[Vector](2)}") + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedScalar.scala b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedScalar.scala new file mode 100644 index 0000000000000..25218a6dcc5af --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedScalar.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql + +// $example on:udf_scalar$ +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions.udf +// $example off:udf_scalar$ + +object UserDefinedScalar { + + def main(args: Array[String]): Unit = { + // $example on:udf_scalar$ + val spark = SparkSession + .builder() + .appName("Spark SQL UDF scalar example") + .getOrCreate() + + // Define and register a zero-argument non-deterministic UDF + // UDF is deterministic by default, i.e. produces the same result for the same input. + val random = udf(() => Math.random()) + spark.udf.register("random", random.asNondeterministic()) + spark.sql("SELECT random()").show() + // +-------+ + // |UDF() | + // +-------+ + // |xxxxxxx| + // +-------+ + + // Define and register a one-argument UDF + val plusOne = udf((x: Int) => x + 1) + spark.udf.register("plusOne", plusOne) + spark.sql("SELECT plusOne(5)").show() + // +------+ + // |UDF(5)| + // +------+ + // | 6| + // +------+ + + // Define a two-argument UDF and register it with Spark in one step + spark.udf.register("strLenScala", (_: String).length + (_: Int)) + spark.sql("SELECT strLenScala('test', 1)").show() + // +--------------------+ + // |strLenScala(test, 1)| + // +--------------------+ + // | 5| + // +--------------------+ + + // UDF in a WHERE clause + spark.udf.register("oneArgFilter", (n: Int) => { n > 5 }) + spark.range(1, 10).createOrReplaceTempView("test") + spark.sql("SELECT * FROM test WHERE oneArgFilter(id)").show() + // +---+ + // | id| + // +---+ + // | 6| + // | 7| + // | 8| + // | 9| + // +---+ + + // $example off:udf_scalar$ + + spark.stop() + } +} diff --git a/external/avro/src/main/java/org/apache/spark/sql/avro/SparkAvroKeyOutputFormat.java b/external/avro/src/main/java/org/apache/spark/sql/avro/SparkAvroKeyOutputFormat.java new file mode 100644 index 0000000000000..55696a6ac2e0f --- /dev/null +++ b/external/avro/src/main/java/org/apache/spark/sql/avro/SparkAvroKeyOutputFormat.java @@ -0,0 +1,94 @@ +/* + * 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.avro; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; + +import org.apache.avro.Schema; +import org.apache.avro.file.CodecFactory; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapreduce.AvroKeyOutputFormat; +import org.apache.avro.mapreduce.Syncable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; + +// A variant of `AvroKeyOutputFormat`, which is used to inject the custom `RecordWriterFactory` so +// that we can set avro file metadata. +public class SparkAvroKeyOutputFormat extends AvroKeyOutputFormat { + public SparkAvroKeyOutputFormat(Map metadata) { + super(new SparkRecordWriterFactory(metadata)); + } + + static class SparkRecordWriterFactory extends RecordWriterFactory { + private final Map metadata; + SparkRecordWriterFactory(Map metadata) { + this.metadata = metadata; + } + + protected RecordWriter, NullWritable> create( + Schema writerSchema, + GenericData dataModel, + CodecFactory compressionCodec, + OutputStream outputStream, + int syncInterval) throws IOException { + return new SparkAvroKeyRecordWriter( + writerSchema, dataModel, compressionCodec, outputStream, syncInterval, metadata); + } + } +} + +// This a fork of org.apache.avro.mapreduce.AvroKeyRecordWriter, in order to set file metadata. +class SparkAvroKeyRecordWriter extends RecordWriter, NullWritable> + implements Syncable { + + private final DataFileWriter mAvroFileWriter; + + SparkAvroKeyRecordWriter( + Schema writerSchema, + GenericData dataModel, + CodecFactory compressionCodec, + OutputStream outputStream, + int syncInterval, + Map metadata) throws IOException { + this.mAvroFileWriter = new DataFileWriter(dataModel.createDatumWriter(writerSchema)); + for (Map.Entry entry : metadata.entrySet()) { + this.mAvroFileWriter.setMeta(entry.getKey(), entry.getValue()); + } + this.mAvroFileWriter.setCodec(compressionCodec); + this.mAvroFileWriter.setSyncInterval(syncInterval); + this.mAvroFileWriter.create(writerSchema, outputStream); + } + + public void write(AvroKey record, NullWritable ignore) throws IOException { + this.mAvroFileWriter.append(record.datum()); + } + + public void close(TaskAttemptContext context) throws IOException { + this.mAvroFileWriter.close(); + } + + public long sync() throws IOException { + return this.mAvroFileWriter.sync(); + } +} diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala index 2c17c16f06da7..8d78cf40a2d58 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala @@ -32,8 +32,10 @@ import org.apache.avro.util.Utf8 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData} -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY +import org.apache.spark.sql.catalyst.util.RebaseDateTime._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String /** @@ -42,6 +44,10 @@ import org.apache.spark.unsafe.types.UTF8String class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { private lazy val decimalConversions = new DecimalConversion() + // Enable rebasing date/timestamp from Julian to Proleptic Gregorian calendar + private val rebaseDateTime = + SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_READ) + private val converter: Any => Any = rootCatalystType match { // A shortcut for empty schema. case st: StructType if st.isEmpty => @@ -88,6 +94,11 @@ class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { case (INT, IntegerType) => (updater, ordinal, value) => updater.setInt(ordinal, value.asInstanceOf[Int]) + case (INT, DateType) if rebaseDateTime => (updater, ordinal, value) => + val days = value.asInstanceOf[Int] + val rebasedDays = rebaseJulianToGregorianDays(days) + updater.setInt(ordinal, rebasedDays) + case (INT, DateType) => (updater, ordinal, value) => updater.setInt(ordinal, value.asInstanceOf[Int]) @@ -95,14 +106,24 @@ class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { updater.setLong(ordinal, value.asInstanceOf[Long]) case (LONG, TimestampType) => avroType.getLogicalType match { - case _: TimestampMillis => (updater, ordinal, value) => - updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) + // For backward compatibility, if the Avro type is Long and it is not logical type + // (the `null` case), the value is processed as timestamp type with millisecond precision. + case null | _: TimestampMillis if rebaseDateTime => (updater, ordinal, value) => + val millis = value.asInstanceOf[Long] + val micros = DateTimeUtils.millisToMicros(millis) + val rebasedMicros = rebaseJulianToGregorianMicros(micros) + updater.setLong(ordinal, rebasedMicros) + case null | _: TimestampMillis => (updater, ordinal, value) => + val millis = value.asInstanceOf[Long] + val micros = DateTimeUtils.millisToMicros(millis) + updater.setLong(ordinal, micros) + case _: TimestampMicros if rebaseDateTime => (updater, ordinal, value) => + val micros = value.asInstanceOf[Long] + val rebasedMicros = rebaseJulianToGregorianMicros(micros) + updater.setLong(ordinal, rebasedMicros) case _: TimestampMicros => (updater, ordinal, value) => - updater.setLong(ordinal, value.asInstanceOf[Long]) - case null => (updater, ordinal, value) => - // For backward compatibility, if the Avro type is Long and it is not logical type, - // the value is processed as timestamp type with millisecond precision. - updater.setLong(ordinal, value.asInstanceOf[Long] * 1000) + val micros = value.asInstanceOf[Long] + updater.setLong(ordinal, micros) case other => throw new IncompatibleSchemaException( s"Cannot convert Avro logical type ${other} to Catalyst Timestamp type.") } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriter.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriter.scala index 06507115f5ed8..2cfa3a4826ed8 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriter.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriter.scala @@ -19,14 +19,17 @@ package org.apache.spark.sql.avro import java.io.{IOException, OutputStream} +import scala.collection.JavaConverters._ + import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.avro.mapred.AvroKey -import org.apache.avro.mapreduce.AvroKeyOutputFormat import org.apache.hadoop.fs.Path import org.apache.hadoop.io.NullWritable import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext} +import org.apache.spark.SPARK_VERSION_SHORT +import org.apache.spark.sql.SPARK_VERSION_METADATA_KEY import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.OutputWriter import org.apache.spark.sql.types._ @@ -45,8 +48,9 @@ private[avro] class AvroOutputWriter( * Overrides the couple of methods responsible for generating the output streams / files so * that the data can be correctly partitioned */ - private val recordWriter: RecordWriter[AvroKey[GenericRecord], NullWritable] = - new AvroKeyOutputFormat[GenericRecord]() { + private val recordWriter: RecordWriter[AvroKey[GenericRecord], NullWritable] = { + val sparkVersion = Map(SPARK_VERSION_METADATA_KEY -> SPARK_VERSION_SHORT).asJava + new SparkAvroKeyOutputFormat(sparkVersion) { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { new Path(path) @@ -57,8 +61,8 @@ private[avro] class AvroOutputWriter( val path = getDefaultWorkFile(context, ".avro") path.getFileSystem(context.getConfiguration).create(path) } - }.getRecordWriter(context) + } override def write(row: InternalRow): Unit = { val key = new AvroKey(serializer.serialize(row).asInstanceOf[GenericRecord]) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala index b7bf7e5543033..daa9c7d611ee4 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala @@ -34,6 +34,9 @@ import org.apache.avro.util.Utf8 import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{SpecializedGetters, SpecificInternalRow} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.RebaseDateTime._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -42,6 +45,10 @@ import org.apache.spark.sql.types._ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) extends Logging { + // Whether to rebase datetimes from Gregorian to Julian calendar in write + private val rebaseDateTime: Boolean = + SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_WRITE) + def serialize(catalystData: Any): Any = { converter.apply(catalystData) } @@ -135,15 +142,24 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: case (BinaryType, BYTES) => (getter, ordinal) => ByteBuffer.wrap(getter.getBinary(ordinal)) + case (DateType, INT) if rebaseDateTime => + (getter, ordinal) => rebaseGregorianToJulianDays(getter.getInt(ordinal)) + case (DateType, INT) => (getter, ordinal) => getter.getInt(ordinal) case (TimestampType, LONG) => avroType.getLogicalType match { - case _: TimestampMillis => (getter, ordinal) => getter.getLong(ordinal) / 1000 + // For backward compatibility, if the Avro type is Long and it is not logical type + // (the `null` case), output the timestamp value as with millisecond precision. + case null | _: TimestampMillis if rebaseDateTime => (getter, ordinal) => + val micros = getter.getLong(ordinal) + val rebasedMicros = rebaseGregorianToJulianMicros(micros) + DateTimeUtils.microsToMillis(rebasedMicros) + case null | _: TimestampMillis => (getter, ordinal) => + DateTimeUtils.microsToMillis(getter.getLong(ordinal)) + case _: TimestampMicros if rebaseDateTime => (getter, ordinal) => + rebaseGregorianToJulianMicros(getter.getLong(ordinal)) case _: TimestampMicros => (getter, ordinal) => getter.getLong(ordinal) - // For backward compatibility, if the Avro type is Long and it is not logical type, - // output the timestamp value as with millisecond precision. - case null => (getter, ordinal) => getter.getLong(ordinal) / 1000 case other => throw new IncompatibleSchemaException( s"Cannot convert Catalyst Timestamp type to Avro logical type ${other}") } diff --git a/external/avro/src/test/resources/before_1582_date_v2_4.avro b/external/avro/src/test/resources/before_1582_date_v2_4.avro new file mode 100644 index 0000000000000..96aa7cbf176a5 Binary files /dev/null and b/external/avro/src/test/resources/before_1582_date_v2_4.avro differ diff --git a/external/avro/src/test/resources/before_1582_ts_micros_v2_4.avro b/external/avro/src/test/resources/before_1582_ts_micros_v2_4.avro new file mode 100644 index 0000000000000..efe5e71a58813 Binary files /dev/null and b/external/avro/src/test/resources/before_1582_ts_micros_v2_4.avro differ diff --git a/external/avro/src/test/resources/before_1582_ts_millis_v2_4.avro b/external/avro/src/test/resources/before_1582_ts_millis_v2_4.avro new file mode 100644 index 0000000000000..dbaec814eb954 Binary files /dev/null and b/external/avro/src/test/resources/before_1582_ts_millis_v2_4.avro differ diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 360160c9c9398..a5224fd104ea9 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -21,7 +21,7 @@ import java.io._ import java.net.URL import java.nio.file.{Files, Paths} import java.sql.{Date, Timestamp} -import java.util.{Locale, TimeZone, UUID} +import java.util.{Locale, UUID} import scala.collection.JavaConverters._ @@ -33,11 +33,12 @@ import org.apache.avro.generic.{GenericData, GenericDatumReader, GenericDatumWri import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed} import org.apache.commons.io.FileUtils -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf, SparkException} import org.apache.spark.sql._ -import org.apache.spark.sql.TestingUDT.{IntervalData, NullData, NullUDT} +import org.apache.spark.sql.TestingUDT.IntervalData import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, UTC} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.{DataSource, FilePartition} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec @@ -83,6 +84,11 @@ abstract class AvroSuite extends QueryTest with SharedSparkSession { }, new GenericDatumReader[Any]()).getSchema.toString(false) } + private def readResourceAvroFile(name: String): DataFrame = { + val url = Thread.currentThread().getContextClassLoader.getResource(name) + spark.read.format("avro").load(url.toString) + } + test("resolve avro data source") { val databricksAvro = "com.databricks.spark.avro" // By default the backward compatibility for com.databricks.spark.avro is enabled. @@ -402,18 +408,19 @@ abstract class AvroSuite extends QueryTest with SharedSparkSession { StructField("float", FloatType, true), StructField("date", DateType, true) )) - TimeZone.setDefault(TimeZone.getTimeZone("UTC")) - val rdd = spark.sparkContext.parallelize(Seq( - Row(1f, null), - Row(2f, new Date(1451948400000L)), - Row(3f, new Date(1460066400500L)) - )) - val df = spark.createDataFrame(rdd, schema) - df.write.format("avro").save(dir.toString) - assert(spark.read.format("avro").load(dir.toString).count == rdd.count) - checkAnswer( - spark.read.format("avro").load(dir.toString).select("date"), - Seq(Row(null), Row(new Date(1451865600000L)), Row(new Date(1459987200000L)))) + withDefaultTimeZone(UTC) { + val rdd = spark.sparkContext.parallelize(Seq( + Row(1f, null), + Row(2f, new Date(1451948400000L)), + Row(3f, new Date(1460066400500L)) + )) + val df = spark.createDataFrame(rdd, schema) + df.write.format("avro").save(dir.toString) + assert(spark.read.format("avro").load(dir.toString).count == rdd.count) + checkAnswer( + spark.read.format("avro").load(dir.toString).select("date"), + Seq(Row(null), Row(new Date(1451865600000L)), Row(new Date(1459987200000L)))) + } } } @@ -1521,6 +1528,110 @@ abstract class AvroSuite extends QueryTest with SharedSparkSession { assert(deprecatedEvents.size === 1) } } + + test("SPARK-31183: compatibility with Spark 2.4 in reading dates/timestamps") { + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_READ.key -> "true") { + checkAnswer( + readResourceAvroFile("before_1582_date_v2_4.avro"), + Row(java.sql.Date.valueOf("1001-01-01"))) + checkAnswer( + readResourceAvroFile("before_1582_ts_micros_v2_4.avro"), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"))) + checkAnswer( + readResourceAvroFile("before_1582_ts_millis_v2_4.avro"), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.124"))) + } + } + + test("SPARK-31183: rebasing microseconds timestamps in write") { + val tsStr = "1001-01-01 01:02:03.123456" + val nonRebased = "1001-01-07 01:09:05.123456" + withTempPath { dir => + val path = dir.getAbsolutePath + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_WRITE.key -> "true") { + Seq(tsStr).toDF("tsS") + .select($"tsS".cast("timestamp").as("ts")) + .write.format("avro") + .save(path) + } + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_READ.key -> "true") { + checkAnswer(spark.read.format("avro").load(path), Row(Timestamp.valueOf(tsStr))) + } + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_READ.key -> "false") { + checkAnswer(spark.read.format("avro").load(path), Row(Timestamp.valueOf(nonRebased))) + } + } + } + + test("SPARK-31183: rebasing milliseconds timestamps in write") { + val tsStr = "1001-01-01 01:02:03.123456" + val rebased = "1001-01-01 01:02:03.123" + val nonRebased = "1001-01-07 01:09:05.123" + Seq( + """{"type": "long","logicalType": "timestamp-millis"}""", + """"long"""").foreach { tsType => + val timestampSchema = s""" + |{ + | "namespace": "logical", + | "type": "record", + | "name": "test", + | "fields": [ + | {"name": "ts", "type": $tsType} + | ] + |}""".stripMargin + withTempPath { dir => + val path = dir.getAbsolutePath + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_WRITE.key -> "true") { + Seq(tsStr).toDF("tsS") + .select($"tsS".cast("timestamp").as("ts")) + .write + .option("avroSchema", timestampSchema) + .format("avro") + .save(path) + } + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_READ.key -> "true") { + checkAnswer( + spark.read.schema("ts timestamp").format("avro").load(path), + Row(Timestamp.valueOf(rebased))) + } + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_READ.key -> "false") { + checkAnswer( + spark.read.schema("ts timestamp").format("avro").load(path), + Row(Timestamp.valueOf(nonRebased))) + } + } + } + } + + test("SPARK-31183: rebasing dates in write") { + withTempPath { dir => + val path = dir.getAbsolutePath + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_WRITE.key -> "true") { + Seq("1001-01-01").toDF("dateS") + .select($"dateS".cast("date").as("date")) + .write.format("avro") + .save(path) + } + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_READ.key -> "true") { + checkAnswer(spark.read.format("avro").load(path), Row(Date.valueOf("1001-01-01"))) + } + withSQLConf(SQLConf.LEGACY_AVRO_REBASE_DATETIME_IN_READ.key -> "false") { + checkAnswer(spark.read.format("avro").load(path), Row(Date.valueOf("1001-01-07"))) + } + } + } + + test("SPARK-31327: Write Spark version into Avro file metadata") { + withTempPath { path => + spark.range(1).repartition(1).write.format("avro").save(path.getCanonicalPath) + val avroFiles = path.listFiles() + .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) + assert(avroFiles.length === 1) + val reader = DataFileReader.openReader(avroFiles(0), new GenericDatumReader[GenericRecord]()) + val version = reader.asInstanceOf[DataFileReader[_]].getMetaString(SPARK_VERSION_METADATA_KEY) + assert(version === SPARK_VERSION_SHORT) + } + } } class AvroV1Suite extends AvroSuite { diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index c357a2ffa0407..3b7bd2a71d2d2 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -50,6 +50,7 @@ com.spotify docker-client test + shaded org.apache.httpcomponents @@ -120,8 +121,8 @@ test - mysql - mysql-connector-java + org.mariadb.jdbc + mariadb-java-client test diff --git a/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh b/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh new file mode 100755 index 0000000000000..00885a3b62327 --- /dev/null +++ b/external/docker-integration-tests/src/test/resources/mariadb_docker_entrypoint.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +dpkg-divert --add /bin/systemctl && ln -sT /bin/true /bin/systemctl +apt update +apt install -y mariadb-plugin-gssapi-server +echo "gssapi_keytab_path=/docker-entrypoint-initdb.d/mariadb.keytab" >> /etc/mysql/mariadb.conf.d/auth_gssapi.cnf +echo "gssapi_principal_name=mariadb/__IP_ADDRESS_REPLACE_ME__@EXAMPLE.COM" >> /etc/mysql/mariadb.conf.d/auth_gssapi.cnf +docker-entrypoint.sh mysqld diff --git a/external/docker-integration-tests/src/test/resources/mariadb_krb_setup.sh b/external/docker-integration-tests/src/test/resources/mariadb_krb_setup.sh new file mode 100755 index 0000000000000..e97be805b4592 --- /dev/null +++ b/external/docker-integration-tests/src/test/resources/mariadb_krb_setup.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +mysql -u root -p'rootpass' -e 'CREATE USER "mariadb/__IP_ADDRESS_REPLACE_ME__@EXAMPLE.COM" IDENTIFIED WITH gssapi;' +mysql -u root -p'rootpass' -D mysql -e 'GRANT ALL PRIVILEGES ON *.* TO "mariadb/__IP_ADDRESS_REPLACE_ME__@EXAMPLE.COM";' diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index cd26fb3628151..376dd4646608c 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -58,10 +58,19 @@ abstract class DatabaseOnDocker { */ def getJdbcUrl(ip: String, port: Int): String + /** + * Optional entry point when container starts + * + * Startup process is a parameter of entry point. This may or may not be considered during + * startup. Prefer entry point to startup process when you need a command always to be executed or + * you want to change the initialization order. + */ + def getEntryPoint: Option[String] = None + /** * Optional process to run when container starts */ - def getStartupProcessName: Option[String] + def getStartupProcessName: Option[String] = None /** * Optional step before container starts @@ -77,6 +86,7 @@ abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventu val db: DatabaseOnDocker private var docker: DockerClient = _ + protected var externalPort: Int = _ private var containerId: String = _ protected var jdbcUrl: String = _ @@ -101,7 +111,7 @@ abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventu docker.pull(db.imageName) } // Configure networking (necessary for boot2docker / Docker Machine) - val externalPort: Int = { + externalPort = { val sock = new ServerSocket(0) val port = sock.getLocalPort sock.close() @@ -118,9 +128,11 @@ abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventu .networkDisabled(false) .env(db.env.map { case (k, v) => s"$k=$v" }.toSeq.asJava) .exposedPorts(s"${db.jdbcPort}/tcp") - if(db.getStartupProcessName.isDefined) { - containerConfigBuilder - .cmd(db.getStartupProcessName.get) + if (db.getEntryPoint.isDefined) { + containerConfigBuilder.entrypoint(db.getEntryPoint.get) + } + if (db.getStartupProcessName.isDefined) { + containerConfigBuilder.cmd(db.getStartupProcessName.get) } db.beforeContainerStart(hostConfigBuilder, containerConfigBuilder) containerConfigBuilder.hostConfig(hostConfigBuilder.build()) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerKrbJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerKrbJDBCIntegrationSuite.scala index 583d8108c716c..009b4a2b1b32e 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerKrbJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerKrbJDBCIntegrationSuite.scala @@ -18,17 +18,22 @@ package org.apache.spark.sql.jdbc import java.io.{File, FileInputStream, FileOutputStream} +import java.sql.Connection +import java.util.Properties import javax.security.auth.login.Configuration import scala.io.Source import org.apache.hadoop.minikdc.MiniKdc +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.StringType import org.apache.spark.util.{SecurityUtils, Utils} abstract class DockerKrbJDBCIntegrationSuite extends DockerJDBCIntegrationSuite { private var kdc: MiniKdc = _ - protected var workDir: File = _ + protected var entryPointDir: File = _ + protected var initDbDir: File = _ protected val userName: String protected var principal: String = _ protected val keytabFileName: String @@ -46,8 +51,9 @@ abstract class DockerKrbJDBCIntegrationSuite extends DockerJDBCIntegrationSuite principal = s"$userName@${kdc.getRealm}" - workDir = Utils.createTempDir() - val keytabFile = new File(workDir, keytabFileName) + entryPointDir = Utils.createTempDir() + initDbDir = Utils.createTempDir() + val keytabFile = new File(initDbDir, keytabFileName) keytabFullPath = keytabFile.getAbsolutePath kdc.createPrincipal(keytabFile, userName) logInfo(s"Created keytab file: $keytabFullPath") @@ -62,6 +68,7 @@ abstract class DockerKrbJDBCIntegrationSuite extends DockerJDBCIntegrationSuite try { if (kdc != null) { kdc.stop() + kdc = null } Configuration.setConfiguration(null) SecurityUtils.setGlobalKrbDebug(false) @@ -71,7 +78,7 @@ abstract class DockerKrbJDBCIntegrationSuite extends DockerJDBCIntegrationSuite } protected def copyExecutableResource( - fileName: String, dir: File, processLine: String => String) = { + fileName: String, dir: File, processLine: String => String = identity) = { val newEntry = new File(dir.getAbsolutePath, fileName) newEntry.createNewFile() Utils.tryWithResource( @@ -91,4 +98,64 @@ abstract class DockerKrbJDBCIntegrationSuite extends DockerJDBCIntegrationSuite logInfo(s"Created executable resource file: ${newEntry.getAbsolutePath}") newEntry } + + override def dataPreparation(conn: Connection): Unit = { + conn.prepareStatement("CREATE TABLE bar (c0 text)").executeUpdate() + conn.prepareStatement("INSERT INTO bar VALUES ('hello')").executeUpdate() + } + + test("Basic read test in query option") { + // This makes sure Spark must do authentication + Configuration.setConfiguration(null) + + val expectedResult = Set("hello").map(Row(_)) + + val query = "SELECT c0 FROM bar" + // query option to pass on the query string. + val df = spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("keytab", keytabFullPath) + .option("principal", principal) + .option("query", query) + .load() + assert(df.collect().toSet === expectedResult) + } + + test("Basic read test in create table path") { + // This makes sure Spark must do authentication + Configuration.setConfiguration(null) + + val expectedResult = Set("hello").map(Row(_)) + + val query = "SELECT c0 FROM bar" + // query option in the create table path. + sql( + s""" + |CREATE OR REPLACE TEMPORARY VIEW queryOption + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$jdbcUrl', query '$query', keytab '$keytabFullPath', principal '$principal') + """.stripMargin.replaceAll("\n", " ")) + assert(sql("select c0 from queryOption").collect().toSet === expectedResult) + } + + test("Basic write test") { + // This makes sure Spark must do authentication + Configuration.setConfiguration(null) + + val props = new Properties + props.setProperty("keytab", keytabFullPath) + props.setProperty("principal", principal) + + val tableName = "write_test" + sqlContext.createDataFrame(Seq(("foo", "bar"))) + .write.jdbc(jdbcUrl, tableName, props) + val df = sqlContext.read.jdbc(jdbcUrl, tableName, props) + + val schema = df.schema + assert(schema.map(_.dataType).toSeq === Seq(StringType, StringType)) + val rows = df.collect() + assert(rows.length === 1) + assert(rows(0).getString(0) === "foo") + assert(rows(0).getString(1) === "bar") + } } diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala new file mode 100644 index 0000000000000..7c1adc990bab3 --- /dev/null +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import javax.security.auth.login.Configuration + +import com.spotify.docker.client.messages.{ContainerConfig, HostConfig} + +import org.apache.spark.sql.execution.datasources.jdbc.connection.SecureConnectionProvider +import org.apache.spark.tags.DockerTest + +@DockerTest +class MariaDBKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { + override protected val userName = s"mariadb/$dockerIp" + override protected val keytabFileName = "mariadb.keytab" + + override val db = new DatabaseOnDocker { + override val imageName = "mariadb:10.4" + override val env = Map( + "MYSQL_ROOT_PASSWORD" -> "rootpass" + ) + override val usesIpc = false + override val jdbcPort = 3306 + + override def getJdbcUrl(ip: String, port: Int): String = + s"jdbc:mysql://$ip:$port/mysql?user=$principal" + + override def getEntryPoint: Option[String] = + Some("/docker-entrypoint/mariadb_docker_entrypoint.sh") + + override def beforeContainerStart( + hostConfigBuilder: HostConfig.Builder, + containerConfigBuilder: ContainerConfig.Builder): Unit = { + def replaceIp(s: String): String = s.replace("__IP_ADDRESS_REPLACE_ME__", dockerIp) + copyExecutableResource("mariadb_docker_entrypoint.sh", entryPointDir, replaceIp) + copyExecutableResource("mariadb_krb_setup.sh", initDbDir, replaceIp) + + hostConfigBuilder.appendBinds( + HostConfig.Bind.from(entryPointDir.getAbsolutePath) + .to("/docker-entrypoint").readOnly(true).build(), + HostConfig.Bind.from(initDbDir.getAbsolutePath) + .to("/docker-entrypoint-initdb.d").readOnly(true).build() + ) + } + } + + override protected def setAuthentication(keytabFile: String, principal: String): Unit = { + val config = new SecureConnectionProvider.JDBCConfiguration( + Configuration.getConfiguration, "Krb5ConnectorContext", keytabFile, principal) + Configuration.setConfiguration(config) + } +} diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 5738307095933..42d64873c44d9 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -37,8 +37,6 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { override def getJdbcUrl(ip: String, port: Int): String = s"jdbc:sqlserver://$ip:$port;user=sa;password=Sapass123;" - - override def getStartupProcessName: Option[String] = None } override def dataPreparation(conn: Connection): Unit = { diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index bba1b5275269b..4cbcb59e02de1 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -35,7 +35,6 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { override val jdbcPort: Int = 3306 override def getJdbcUrl(ip: String, port: Int): String = s"jdbc:mysql://$ip:$port/mysql?user=root&password=rootpass" - override def getStartupProcessName: Option[String] = None } override def dataPreparation(conn: Connection): Unit = { diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 6faa888cf18ed..24c3adb9c0153 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -66,7 +66,6 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSpark override val jdbcPort: Int = 1521 override def getJdbcUrl(ip: String, port: Int): String = s"jdbc:oracle:thin:system/oracle@//$ip:$port/xe" - override def getStartupProcessName: Option[String] = None } override def dataPreparation(conn: Connection): Unit = { 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 599f00def0750..6611bc2d19ed8 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 @@ -37,7 +37,6 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val jdbcPort = 5432 override def getJdbcUrl(ip: String, port: Int): String = s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass" - override def getStartupProcessName: Option[String] = None } override def dataPreparation(conn: Connection): Unit = { diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala index 721a4882b986a..adf30fbdc1e12 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala @@ -17,15 +17,11 @@ package org.apache.spark.sql.jdbc -import java.sql.Connection -import java.util.Properties import javax.security.auth.login.Configuration import com.spotify.docker.client.messages.{ContainerConfig, HostConfig} -import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.datasources.jdbc.connection.PostgresConnectionProvider -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.execution.datasources.jdbc.connection.SecureConnectionProvider import org.apache.spark.tags.DockerTest @DockerTest @@ -44,86 +40,22 @@ class PostgresKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override def getJdbcUrl(ip: String, port: Int): String = s"jdbc:postgresql://$ip:$port/postgres?user=$principal&gsslib=gssapi" - override def getStartupProcessName: Option[String] = None - override def beforeContainerStart( hostConfigBuilder: HostConfig.Builder, containerConfigBuilder: ContainerConfig.Builder): Unit = { def replaceIp(s: String): String = s.replace("__IP_ADDRESS_REPLACE_ME__", dockerIp) - copyExecutableResource("postgres_krb_setup.sh", workDir, replaceIp) + copyExecutableResource("postgres_krb_setup.sh", initDbDir, replaceIp) hostConfigBuilder.appendBinds( - HostConfig.Bind.from(workDir.getAbsolutePath) + HostConfig.Bind.from(initDbDir.getAbsolutePath) .to("/docker-entrypoint-initdb.d").readOnly(true).build() ) } } override protected def setAuthentication(keytabFile: String, principal: String): Unit = { - val config = new PostgresConnectionProvider.PGJDBCConfiguration( + val config = new SecureConnectionProvider.JDBCConfiguration( Configuration.getConfiguration, "pgjdbc", keytabFile, principal) Configuration.setConfiguration(config) } - - override def dataPreparation(conn: Connection): Unit = { - conn.prepareStatement("CREATE DATABASE foo").executeUpdate() - conn.setCatalog("foo") - conn.prepareStatement("CREATE TABLE bar (c0 text)").executeUpdate() - conn.prepareStatement("INSERT INTO bar VALUES ('hello')").executeUpdate() - } - - test("Basic read test in query option") { - // This makes sure Spark must do authentication - Configuration.setConfiguration(null) - - val expectedResult = Set("hello").map(Row(_)) - - val query = "SELECT c0 FROM bar" - // query option to pass on the query string. - val df = spark.read.format("jdbc") - .option("url", jdbcUrl) - .option("keytab", keytabFullPath) - .option("principal", principal) - .option("query", query) - .load() - assert(df.collect().toSet === expectedResult) - } - - test("Basic read test in create table path") { - // This makes sure Spark must do authentication - Configuration.setConfiguration(null) - - val expectedResult = Set("hello").map(Row(_)) - - val query = "SELECT c0 FROM bar" - // query option in the create table path. - sql( - s""" - |CREATE OR REPLACE TEMPORARY VIEW queryOption - |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$jdbcUrl', query '$query', keytab '$keytabFullPath', principal '$principal') - """.stripMargin.replaceAll("\n", " ")) - assert(sql("select c0 from queryOption").collect().toSet === expectedResult) - } - - test("Basic write test") { - // This makes sure Spark must do authentication - Configuration.setConfiguration(null) - - val props = new Properties - props.setProperty("keytab", keytabFullPath) - props.setProperty("principal", principal) - - val tableName = "write_test" - sqlContext.createDataFrame(Seq(("foo", "bar"))) - .write.jdbc(jdbcUrl, tableName, props) - val df = sqlContext.read.jdbc(jdbcUrl, tableName, props) - - val schema = df.schema - assert(schema.map(_.dataType).toSeq === Seq(StringType, StringType)) - val rows = df.collect() - assert(rows.length === 1) - assert(rows(0).getString(0) === "foo") - assert(rows(0).getString(1) === "bar") - } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala index 460bb8bd34ec6..b7e42c00b8cff 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala @@ -29,6 +29,7 @@ package object kafka010 { // scalastyle:ignore private[kafka010] val PRODUCER_CACHE_TIMEOUT = ConfigBuilder("spark.kafka.producer.cache.timeout") .doc("The expire time to remove the unused producers.") + .version("2.2.1") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("10m") @@ -36,6 +37,7 @@ package object kafka010 { // scalastyle:ignore ConfigBuilder("spark.kafka.producer.cache.evictorThreadRunInterval") .doc("The interval of time between runs of the idle evictor thread for producer pool. " + "When non-positive, no idle evictor thread will be run.") + .version("3.0.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("1m") @@ -43,12 +45,14 @@ package object kafka010 { // scalastyle:ignore ConfigBuilder("spark.kafka.consumer.cache.capacity") .doc("The maximum number of consumers cached. Please note it's a soft limit" + " (check Structured Streaming Kafka integration guide for further details).") + .version("3.0.0") .intConf .createWithDefault(64) private[kafka010] val CONSUMER_CACHE_JMX_ENABLED = ConfigBuilder("spark.kafka.consumer.cache.jmx.enable") .doc("Enable or disable JMX for pools created with this configuration instance.") + .version("3.0.0") .booleanConf .createWithDefault(false) @@ -57,6 +61,7 @@ package object kafka010 { // scalastyle:ignore .doc("The minimum amount of time a consumer may sit idle in the pool before " + "it is eligible for eviction by the evictor. " + "When non-positive, no consumers will be evicted from the pool due to idle time alone.") + .version("3.0.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("5m") @@ -64,6 +69,7 @@ package object kafka010 { // scalastyle:ignore ConfigBuilder("spark.kafka.consumer.cache.evictorThreadRunInterval") .doc("The interval of time between runs of the idle evictor thread for consumer pool. " + "When non-positive, no idle evictor thread will be run.") + .version("3.0.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("1m") @@ -72,6 +78,7 @@ package object kafka010 { // scalastyle:ignore .doc("The minimum amount of time a fetched data may sit idle in the pool before " + "it is eligible for eviction by the evictor. " + "When non-positive, no fetched data will be evicted from the pool due to idle time alone.") + .version("3.0.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("5m") @@ -79,6 +86,7 @@ package object kafka010 { // scalastyle:ignore ConfigBuilder("spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval") .doc("The interval of time between runs of the idle evictor thread for fetched data pool. " + "When non-positive, no idle evictor thread will be run.") + .version("3.0.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("1m") } diff --git a/external/kafka-0-10-sql/src/test/resources/log4j.properties b/external/kafka-0-10-sql/src/test/resources/log4j.properties index 75e3b53a093f6..daf05723d4e7b 100644 --- a/external/kafka-0-10-sql/src/test/resources/log4j.properties +++ b/external/kafka-0-10-sql/src/test/resources/log4j.properties @@ -25,4 +25,6 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{ # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.spark-project.jetty=WARN - +log4j.logger.org.apache.spark.sql.kafka010.KafkaTestUtils=DEBUG +log4j.logger.org.apache.directory.server.kerberos.kdc.authentication=DEBUG +log4j.logger.org.apache.directory.server.core.DefaultDirectoryService=DEBUG diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala index 79239e57409e9..702bd4f6ebdad 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala @@ -62,7 +62,7 @@ class KafkaDelegationTokenSuite extends StreamTest with SharedSparkSession with } } - ignore("Roundtrip") { + testRetry("Roundtrip", 3) { val hadoopConf = new Configuration() val manager = new HadoopDelegationTokenManager(spark.sparkContext.conf, hadoopConf, null) val credentials = new Credentials() diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 5c8c5b1f3b307..4e808a5277a98 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -314,7 +314,7 @@ class KafkaSinkMicroBatchStreamingSuite extends KafkaSinkStreamingSuiteBase { try { input.addData("1", "2", "3") verifyResult(writer) { - assert(writer.lastProgress.sink.numOutputRows == 3L) + assert(writer.recentProgress.exists(_.sink.numOutputRows == 3L)) } } finally { writer.stop() diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 90fb188bba21e..4f846199cfbc7 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -168,6 +168,7 @@ class KafkaTestUtils( kdc.getKrb5conf.delete() Files.write(krb5confStr, kdc.getKrb5conf, StandardCharsets.UTF_8) + logDebug(s"krb5.conf file content: $krb5confStr") } private def addedKrb5Config(key: String, value: String): String = { @@ -299,6 +300,7 @@ class KafkaTestUtils( } brokerReady = false zkReady = false + kdcReady = false if (producer != null) { producer.close() @@ -307,6 +309,7 @@ class KafkaTestUtils( if (adminClient != null) { adminClient.close() + adminClient = null } if (server != null) { @@ -341,6 +344,7 @@ class KafkaTestUtils( Configuration.getConfiguration.refresh() if (kdc != null) { kdc.stop() + kdc = null } UserGroupInformation.reset() SecurityUtils.setGlobalKrbDebug(false) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala index 3d2921fa900a3..0679a496fc3c7 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala @@ -26,43 +26,51 @@ package object kafka010 { //scalastyle:ignore private[spark] val CONSUMER_CACHE_ENABLED = ConfigBuilder("spark.streaming.kafka.consumer.cache.enabled") + .version("2.2.1") .booleanConf .createWithDefault(true) private[spark] val CONSUMER_POLL_MS = ConfigBuilder("spark.streaming.kafka.consumer.poll.ms") - .longConf - .createOptional + .version("2.0.1") + .longConf + .createOptional private[spark] val CONSUMER_CACHE_INITIAL_CAPACITY = ConfigBuilder("spark.streaming.kafka.consumer.cache.initialCapacity") - .intConf - .createWithDefault(16) + .version("2.0.1") + .intConf + .createWithDefault(16) private[spark] val CONSUMER_CACHE_MAX_CAPACITY = ConfigBuilder("spark.streaming.kafka.consumer.cache.maxCapacity") - .intConf - .createWithDefault(64) + .version("2.0.1") + .intConf + .createWithDefault(64) private[spark] val CONSUMER_CACHE_LOAD_FACTOR = ConfigBuilder("spark.streaming.kafka.consumer.cache.loadFactor") - .doubleConf - .createWithDefault(0.75) + .version("2.0.1") + .doubleConf + .createWithDefault(0.75) private[spark] val MAX_RATE_PER_PARTITION = ConfigBuilder("spark.streaming.kafka.maxRatePerPartition") - .longConf - .createWithDefault(0) + .version("1.3.0") + .longConf + .createWithDefault(0) private[spark] val MIN_RATE_PER_PARTITION = ConfigBuilder("spark.streaming.kafka.minRatePerPartition") - .longConf - .createWithDefault(1) + .version("2.4.0") + .longConf + .createWithDefault(1) private[spark] val ALLOW_NON_CONSECUTIVE_OFFSETS = ConfigBuilder("spark.streaming.kafka.allowNonConsecutiveOffsets") - .booleanConf - .createWithDefault(false) + .version("2.3.1") + .booleanConf + .createWithDefault(false) } diff --git a/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py b/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py index 49794faab88c4..777a33270c415 100644 --- a/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py +++ b/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py @@ -26,6 +26,7 @@ name of the Kinesis stream (ie. mySparkStream) endpoint of the Kinesis service (e.g. https://kinesis.us-east-1.amazonaws.com) + region name of the Kinesis endpoint (e.g. us-east-1) Example: @@ -34,10 +35,10 @@ $ export AWS_SECRET_KEY= # run the example - $ bin/spark-submit -jars external/kinesis-asl/target/scala-*/\ - spark-streaming-kinesis-asl-assembly_*.jar \ + $ bin/spark-submit --jars \ + 'external/kinesis-asl-assembly/target/spark-streaming-kinesis-asl-assembly_*.jar' \ external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py \ - myAppName mySparkStream https://kinesis.us-east-1.amazonaws.com + myAppName mySparkStream https://kinesis.us-east-1.amazonaws.com us-east-1 There is a companion helper class called KinesisWordProducerASL which puts dummy data onto the Kinesis stream. 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 a5d5ac769b28d..32f4a6759474f 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 @@ -78,7 +78,7 @@ object KinesisWordCountASL extends Logging { if (args.length != 3) { System.err.println( """ - |Usage: KinesisWordCountASL + |Usage: KinesisWordCountASL | | is the name of the consumer app, used to track the read data in DynamoDB | is the name of the Kinesis stream @@ -171,11 +171,11 @@ object KinesisWordCountASL extends Logging { * is the endpoint of the Kinesis service * (ie. https://kinesis.us-east-1.amazonaws.com) * is the rate of records per second to put onto the stream - * is the rate of records per second to put onto the stream + * is the number of words per record * * Example: * $ SPARK_HOME/bin/run-example streaming.KinesisWordProducerASL mySparkStream \ - * https://kinesis.us-east-1.amazonaws.com us-east-1 10 5 + * https://kinesis.us-east-1.amazonaws.com 10 5 */ object KinesisWordProducerASL { def main(args: Array[String]): Unit = { @@ -183,13 +183,13 @@ object KinesisWordProducerASL { System.err.println( """ |Usage: KinesisWordProducerASL - + | | | is the name of the Kinesis stream | is the endpoint of the Kinesis service | (e.g. https://kinesis.us-east-1.amazonaws.com) | is the rate of records per second to put onto the stream - | is the rate of records per second to put onto the stream + | is the number of words per record | """.stripMargin) diff --git a/licenses-binary/LICENSE-vis-timeline.txt b/licenses-binary/LICENSE-vis-timeline.txt new file mode 100644 index 0000000000000..d343783c16791 --- /dev/null +++ b/licenses-binary/LICENSE-vis-timeline.txt @@ -0,0 +1,23 @@ +vis-timeline and vis-graph2d +https://visjs.github.io/vis-timeline/ + +Create a fully customizable, interactive timeline with items and ranges. + +@version 7.3.4 +@date 2020-03-18T17:03:58.105Z + +@copyright (c) 2011-2017 Almende B.V, http://almende.com +@copyright (c) 2017-2019 visjs contributors, https://github.com/visjs + +@license +vis.js is dual licensed under both + + 1. The Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0 + + and + + 2. The MIT License + http://opensource.org/licenses/MIT + +vis.js may be distributed under either license. diff --git a/licenses-binary/LICENSE-vis.txt b/licenses-binary/LICENSE-vis.txt deleted file mode 100644 index 18b7323059a41..0000000000000 --- a/licenses-binary/LICENSE-vis.txt +++ /dev/null @@ -1,22 +0,0 @@ -vis.js -https://github.com/almende/vis - -A dynamic, browser-based visualization library. - -@version 4.16.1 -@date 2016-04-18 - -@license -Copyright (C) 2011-2016 Almende B.V, http://almende.com - -Vis.js is dual licensed under both - -* The Apache 2.0 License - http://www.apache.org/licenses/LICENSE-2.0 - -and - -* The MIT License - http://opensource.org/licenses/MIT - -Vis.js may be distributed under either license. \ No newline at end of file diff --git a/licenses/LICENSE-vis-timeline.txt b/licenses/LICENSE-vis-timeline.txt new file mode 100644 index 0000000000000..d343783c16791 --- /dev/null +++ b/licenses/LICENSE-vis-timeline.txt @@ -0,0 +1,23 @@ +vis-timeline and vis-graph2d +https://visjs.github.io/vis-timeline/ + +Create a fully customizable, interactive timeline with items and ranges. + +@version 7.3.4 +@date 2020-03-18T17:03:58.105Z + +@copyright (c) 2011-2017 Almende B.V, http://almende.com +@copyright (c) 2017-2019 visjs contributors, https://github.com/visjs + +@license +vis.js is dual licensed under both + + 1. The Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0 + + and + + 2. The MIT License + http://opensource.org/licenses/MIT + +vis.js may be distributed under either license. diff --git a/licenses/LICENSE-vis.txt b/licenses/LICENSE-vis.txt deleted file mode 100644 index 18b7323059a41..0000000000000 --- a/licenses/LICENSE-vis.txt +++ /dev/null @@ -1,22 +0,0 @@ -vis.js -https://github.com/almende/vis - -A dynamic, browser-based visualization library. - -@version 4.16.1 -@date 2016-04-18 - -@license -Copyright (C) 2011-2016 Almende B.V, http://almende.com - -Vis.js is dual licensed under both - -* The Apache 2.0 License - http://www.apache.org/licenses/LICENSE-2.0 - -and - -* The MIT License - http://opensource.org/licenses/MIT - -Vis.js may be distributed under either license. \ No newline at end of file diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala index ea1219165b980..3d3e7a22e594b 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala @@ -27,8 +27,9 @@ private[spark] object BLAS extends Serializable { @transient private var _f2jBLAS: NetlibBLAS = _ @transient private var _nativeBLAS: NetlibBLAS = _ + private val nativeL1Threshold: Int = 256 - // For level-1 routines, we use Java implementation. + // For level-1 function dspmv, use f2jBLAS for better performance. private[ml] def f2jBLAS: NetlibBLAS = { if (_f2jBLAS == null) { _f2jBLAS = new F2jBLAS @@ -36,6 +37,14 @@ private[spark] object BLAS extends Serializable { _f2jBLAS } + private[ml] def getBLAS(vectorSize: Int): NetlibBLAS = { + if (vectorSize < nativeL1Threshold) { + f2jBLAS + } else { + nativeBLAS + } + } + /** * y += a * x */ @@ -63,7 +72,7 @@ private[spark] object BLAS extends Serializable { */ private def axpy(a: Double, x: DenseVector, y: DenseVector): Unit = { val n = x.size - f2jBLAS.daxpy(n, a, x.values, 1, y.values, 1) + getBLAS(n).daxpy(n, a, x.values, 1, y.values, 1) } /** @@ -94,7 +103,7 @@ private[spark] object BLAS extends Serializable { private[spark] def axpy(a: Double, X: DenseMatrix, Y: DenseMatrix): Unit = { require(X.numRows == Y.numRows && X.numCols == Y.numCols, "Dimension mismatch: " + s"size(X) = ${(X.numRows, X.numCols)} but size(Y) = ${(Y.numRows, Y.numCols)}.") - f2jBLAS.daxpy(X.numRows * X.numCols, a, X.values, 1, Y.values, 1) + getBLAS(X.values.length).daxpy(X.numRows * X.numCols, a, X.values, 1, Y.values, 1) } /** @@ -123,7 +132,7 @@ private[spark] object BLAS extends Serializable { */ private def dot(x: DenseVector, y: DenseVector): Double = { val n = x.size - f2jBLAS.ddot(n, x.values, 1, y.values, 1) + getBLAS(n).ddot(n, x.values, 1, y.values, 1) } /** @@ -218,16 +227,16 @@ private[spark] object BLAS extends Serializable { def scal(a: Double, x: Vector): Unit = { x match { case sx: SparseVector => - f2jBLAS.dscal(sx.values.length, a, sx.values, 1) + getBLAS(sx.values.length).dscal(sx.values.length, a, sx.values, 1) case dx: DenseVector => - f2jBLAS.dscal(dx.values.length, a, dx.values, 1) + getBLAS(dx.size).dscal(dx.values.length, a, dx.values, 1) case _ => throw new IllegalArgumentException(s"scal doesn't support vector type ${x.getClass}.") } } // For level-3 routines, we use the native BLAS. - private def nativeBLAS: NetlibBLAS = { + private[ml] def nativeBLAS: NetlibBLAS = { if (_nativeBLAS == null) { _nativeBLAS = NativeBLAS } @@ -374,7 +383,7 @@ private[spark] object BLAS extends Serializable { // gemm: alpha is equal to 0 and beta is equal to 1. Returning C. return } else if (alpha == 0.0) { - f2jBLAS.dscal(C.values.length, beta, C.values, 1) + getBLAS(C.values.length).dscal(C.values.length, beta, C.values, 1) } else { A match { case sparse: SparseMatrix => gemm(alpha, sparse, B, beta, C) @@ -480,7 +489,7 @@ private[spark] object BLAS extends Serializable { } else { // Scale matrix first if `beta` is not equal to 1.0 if (beta != 1.0) { - f2jBLAS.dscal(C.values.length, beta, C.values, 1) + getBLAS(C.values.length).dscal(C.values.length, beta, C.values, 1) } // Perform matrix multiplication and add to C. The rows of A are multiplied by the columns of // B, and added to C. diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala index 877ac68983348..781f3da313d82 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala @@ -23,6 +23,12 @@ import org.apache.spark.ml.util.TestingUtils._ class BLASSuite extends SparkMLFunSuite { + test("nativeL1Threshold") { + assert(getBLAS(128) == BLAS.f2jBLAS) + assert(getBLAS(256) == BLAS.nativeBLAS) + assert(getBLAS(512) == BLAS.nativeBLAS) + } + test("copy") { val sx = Vectors.sparse(4, Array(0, 2), Array(1.0, -2.0)) val dx = Vectors.dense(1.0, 0.0, -2.0, 0.0) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala index 1247882d6c1bd..8815eb29bc860 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -19,15 +19,13 @@ package org.apache.spark.ml import scala.annotation.varargs -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.param.{ParamMap, ParamPair} import org.apache.spark.sql.Dataset /** - * :: DeveloperApi :: * Abstract class for estimators that fit models to data. */ -@DeveloperApi abstract class Estimator[M <: Model[M]] extends PipelineStage { /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/Model.scala b/mllib/src/main/scala/org/apache/spark/ml/Model.scala index c581fed177273..709a1124cbb5e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Model.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Model.scala @@ -17,16 +17,13 @@ package org.apache.spark.ml -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.param.ParamMap /** - * :: DeveloperApi :: * A fitted model, i.e., a [[Transformer]] produced by an [[Estimator]]. * * @tparam M model type */ -@DeveloperApi abstract class Model[M <: Model[M]] extends Transformer { /** * The parent estimator that produced this model. 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 d44031fc7a11d..0d2ede5e66805 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -36,14 +36,11 @@ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType /** - * :: DeveloperApi :: * A stage in a pipeline, either an [[Estimator]] or a [[Transformer]]. */ -@DeveloperApi abstract class PipelineStage extends Params with Logging { /** - * :: DeveloperApi :: * * Check transform validity and derive the output schema from the input schema. * @@ -54,7 +51,6 @@ abstract class PipelineStage extends Params with Logging { * Typical implementation should first conduct verification on schema change and parameter * validity, including complex parameter interaction checks. */ - @DeveloperApi def transformSchema(schema: StructType): StructType /** 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 98dd692cbe55d..febeba7e13fcb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ @@ -99,7 +99,6 @@ 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()`. If this predictor supports * weights, it accepts all NumericType weights, which will be automatically casted to DoubleType @@ -112,7 +111,6 @@ private[ml] trait PredictorParams extends Params * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. */ -@DeveloperApi abstract class Predictor[ FeaturesType, Learner <: Predictor[FeaturesType, Learner, M], @@ -190,7 +188,6 @@ abstract class Predictor[ } /** - * :: DeveloperApi :: * Abstraction for a model for prediction tasks (regression and classification). * * @tparam FeaturesType Type of features. @@ -198,7 +195,6 @@ abstract class Predictor[ * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. */ -@DeveloperApi abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] extends Model[M] with PredictorParams { diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 1652131a9003a..3b60b5ae294ea 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml import scala.annotation.varargs import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ @@ -29,10 +29,8 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ /** - * :: DeveloperApi :: * Abstract class for transformers that transform one dataset into another. */ -@DeveloperApi abstract class Transformer extends PipelineStage { /** @@ -75,11 +73,9 @@ abstract class Transformer extends PipelineStage { } /** - * :: DeveloperApi :: * Abstract class for transformers that take one input column, apply transformation, and output the * result as a new column. */ -@DeveloperApi abstract class UnaryTransformer[IN: TypeTag, OUT: TypeTag, T <: UnaryTransformer[IN, OUT, T]] extends Transformer with HasInputCol with HasOutputCol with Logging { diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala index b3d24b70fc55e..91dd6abae833b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -19,12 +19,10 @@ package org.apache.spark.ml.attribute import scala.collection.mutable.ArrayBuffer -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.{Metadata, MetadataBuilder, StructField} /** - * :: DeveloperApi :: * Attributes that describe a vector ML column. * * @param name name of the attribute group (the ML column name) @@ -33,7 +31,6 @@ import org.apache.spark.sql.types.{Metadata, MetadataBuilder, StructField} * @param attrs optional array of attributes. Attribute will be copied with their corresponding * indices in the array. */ -@DeveloperApi class AttributeGroup private ( val name: String, val numAttributes: Option[Int], @@ -186,10 +183,8 @@ class AttributeGroup private ( } /** - * :: DeveloperApi :: * Factory methods to create attribute groups. */ -@DeveloperApi object AttributeGroup { import AttributeKeys._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala index 078fecf088282..f1f3a97f8ac1c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala @@ -17,20 +17,13 @@ package org.apache.spark.ml.attribute -import org.apache.spark.annotation.DeveloperApi /** - * :: DeveloperApi :: * An enum-like type for attribute types: [[AttributeType$#Numeric]], [[AttributeType$#Nominal]], * and [[AttributeType$#Binary]]. */ -@DeveloperApi sealed abstract class AttributeType(val name: String) -/** - * :: DeveloperApi :: - */ -@DeveloperApi object AttributeType { /** Numeric type. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala index 756dd677cb579..576c771d83bec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala @@ -19,14 +19,11 @@ package org.apache.spark.ml.attribute import scala.annotation.varargs -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.types.{DoubleType, Metadata, MetadataBuilder, NumericType, StructField} /** - * :: DeveloperApi :: * Abstract class for ML attributes. */ -@DeveloperApi sealed abstract class Attribute extends Serializable { name.foreach { n => @@ -150,10 +147,6 @@ private[attribute] trait AttributeFactory { def fromStructField(field: StructField): Attribute = decodeStructField(field, false) } -/** - * :: DeveloperApi :: - */ -@DeveloperApi object Attribute extends AttributeFactory { private[attribute] override def fromMetadata(metadata: Metadata): Attribute = { @@ -182,7 +175,6 @@ object Attribute extends AttributeFactory { /** - * :: DeveloperApi :: * A numeric attribute with optional summary statistics. * @param name optional name * @param index optional index @@ -191,7 +183,6 @@ object Attribute extends AttributeFactory { * @param std optional standard deviation * @param sparsity optional sparsity (ratio of zeros) */ -@DeveloperApi class NumericAttribute private[ml] ( override val name: Option[String] = None, override val index: Option[Int] = None, @@ -299,10 +290,8 @@ class NumericAttribute private[ml] ( } /** - * :: DeveloperApi :: * Factory methods for numeric attributes. */ -@DeveloperApi object NumericAttribute extends AttributeFactory { /** The default numeric attribute. */ @@ -321,7 +310,6 @@ object NumericAttribute extends AttributeFactory { } /** - * :: DeveloperApi :: * A nominal attribute. * @param name optional name * @param index optional index @@ -330,7 +318,6 @@ object NumericAttribute extends AttributeFactory { * defined. * @param values optional values. At most one of `numValues` and `values` can be defined. */ -@DeveloperApi class NominalAttribute private[ml] ( override val name: Option[String] = None, override val index: Option[Int] = None, @@ -464,10 +451,8 @@ class NominalAttribute private[ml] ( } /** - * :: DeveloperApi :: * Factory methods for nominal attributes. */ -@DeveloperApi object NominalAttribute extends AttributeFactory { /** The default nominal attribute. */ @@ -487,13 +472,11 @@ object NominalAttribute extends AttributeFactory { } /** - * :: DeveloperApi :: * A binary attribute. * @param name optional name * @param index optional index * @param values optional values. If set, its size must be 2. */ -@DeveloperApi class BinaryAttribute private[ml] ( override val name: Option[String] = None, override val index: Option[Int] = None, @@ -566,10 +549,8 @@ class BinaryAttribute private[ml] ( } /** - * :: DeveloperApi :: * Factory methods for binary attributes. */ -@DeveloperApi object BinaryAttribute extends AttributeFactory { /** The default binary attribute. */ @@ -586,10 +567,8 @@ object BinaryAttribute extends AttributeFactory { } /** - * :: DeveloperApi :: * An unresolved attribute. */ -@DeveloperApi object UnresolvedAttribute extends Attribute { override def attrType: AttributeType = AttributeType.Unresolved 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 ac92299a931bc..1f3f291644f93 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 @@ -18,7 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkException -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, VectorUDT} @@ -62,8 +62,6 @@ private[spark] trait ClassifierParams } /** - * :: DeveloperApi :: - * * Single-label binary or multiclass classification. * Classes are indexed {0, 1, ..., numClasses - 1}. * @@ -71,7 +69,6 @@ private[spark] trait ClassifierParams * @tparam E Concrete Estimator type * @tparam M Concrete Model type */ -@DeveloperApi abstract class Classifier[ FeaturesType, E <: Classifier[FeaturesType, E, M], @@ -166,15 +163,12 @@ abstract class Classifier[ } /** - * :: DeveloperApi :: - * * Model produced by a [[Classifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam M Concrete Model type */ -@DeveloperApi abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[FeaturesType, M]] extends PredictionModel[FeaturesType, M] with ClassifierParams { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index cc5da3446f1f8..9758e3ca72c38 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.{DenseVector, Vector, VectorUDT} import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils @@ -41,15 +41,12 @@ private[ml] trait ProbabilisticClassifierParams /** - * :: DeveloperApi :: - * * Single-label binary or multiclass classifier which can output class conditional probabilities. * * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam E Concrete Estimator type * @tparam M Concrete Model type */ -@DeveloperApi abstract class ProbabilisticClassifier[ FeaturesType, E <: ProbabilisticClassifier[FeaturesType, E, M], @@ -65,15 +62,12 @@ abstract class ProbabilisticClassifier[ /** - * :: DeveloperApi :: - * * Model produced by a [[ProbabilisticClassifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam M Concrete Model type */ -@DeveloperApi abstract class ProbabilisticClassificationModel[ FeaturesType, M <: ProbabilisticClassificationModel[FeaturesType, M]] 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 e30be8c20dcc3..4e86b3b247ace 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 @@ -26,7 +26,7 @@ import org.json4s.DefaultFormats import org.json4s.JsonAST.JObject import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg._ @@ -762,8 +762,6 @@ class DistributedLDAModel private[ml] ( private var _checkpointFiles: Array[String] = oldDistributedModel.checkpointFiles /** - * :: DeveloperApi :: - * * If using checkpointing and `LDA.keepLastCheckpoint` is set to true, then there may be * saved checkpoint files. This method is provided so that users can manage those files. * @@ -773,18 +771,14 @@ class DistributedLDAModel private[ml] ( * * @return Checkpoint files from training */ - @DeveloperApi @Since("2.0.0") def getCheckpointFiles: Array[String] = _checkpointFiles /** - * :: DeveloperApi :: - * * Remove any remaining checkpoint files from training. * * @see [[getCheckpointFiles]] */ - @DeveloperApi @Since("2.0.0") def deleteCheckpointFiles(): Unit = { val hadoopConf = sparkSession.sparkContext.hadoopConfiguration diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala index e7b949ddce344..f7a2d0dbe998f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala @@ -17,16 +17,14 @@ package org.apache.spark.ml.evaluation -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.param.{ParamMap, Params} import org.apache.spark.sql.Dataset /** - * :: DeveloperApi :: * Abstract class for evaluators that compute metrics from predictions. */ @Since("1.5.0") -@DeveloperApi abstract class Evaluator extends Params { /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ANOVASelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ANOVASelector.scala new file mode 100644 index 0000000000000..40f7c8da47a84 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ANOVASelector.scala @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import scala.collection.mutable.ArrayBuilder + +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since +import org.apache.spark.ml._ +import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param._ +import org.apache.spark.ml.stat.{ANOVATest, SelectionTestResult} +import org.apache.spark.ml.util._ +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} + + +/** + * ANOVA F-value Classification selector, which selects continuous features to use for predicting a + * categorical label. + * The selector supports different selection methods: `numTopFeatures`, `percentile`, `fpr`, + * `fdr`, `fwe`. + * - `numTopFeatures` chooses a fixed number of top features according to a F value classification + * test. + * - `percentile` is similar but chooses a fraction of all features instead of a fixed number. + * - `fpr` chooses all features whose p-value are below a threshold, thus controlling the false + * positive rate of selection. + * - `fdr` uses the [Benjamini-Hochberg procedure] + * (https://en.wikipedia.org/wiki/False_discovery_rate#Benjamini.E2.80.93Hochberg_procedure) + * to choose all features whose false discovery rate is below a threshold. + * - `fwe` chooses all features whose p-values are below a threshold. The threshold is scaled by + * 1/numFeatures, thus controlling the family-wise error rate of selection. + * By default, the selection method is `numTopFeatures`, with the default number of top features + * set to 50. + */ +@Since("3.1.0") +final class ANOVASelector @Since("3.1.0")(@Since("3.1.0") override val uid: String) + extends Estimator[ANOVASelectorModel] with FValueSelectorParams + with DefaultParamsWritable { + + @Since("3.1.0") + def this() = this(Identifiable.randomUID("ANOVASelector")) + + /** @group setParam */ + @Since("3.1.0") + def setNumTopFeatures(value: Int): this.type = set(numTopFeatures, value) + + /** @group setParam */ + @Since("3.1.0") + def setPercentile(value: Double): this.type = set(percentile, value) + + /** @group setParam */ + @Since("3.1.0") + def setFpr(value: Double): this.type = set(fpr, value) + + /** @group setParam */ + @Since("3.1.0") + def setFdr(value: Double): this.type = set(fdr, value) + + /** @group setParam */ + @Since("3.1.0") + def setFwe(value: Double): this.type = set(fwe, value) + + /** @group setParam */ + @Since("3.1.0") + def setSelectorType(value: String): this.type = set(selectorType, value) + + /** @group setParam */ + @Since("3.1.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("3.1.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + + /** @group setParam */ + @Since("3.1.0") + def setLabelCol(value: String): this.type = set(labelCol, value) + + @Since("3.1.0") + override def fit(dataset: Dataset[_]): ANOVASelectorModel = { + transformSchema(dataset.schema, logging = true) + + val testResult = ANOVATest.testClassification(dataset, getFeaturesCol, getLabelCol) + .zipWithIndex + val features = $(selectorType) match { + case "numTopFeatures" => + testResult + .sortBy { case (res, _) => res.pValue } + .take(getNumTopFeatures) + case "percentile" => + testResult + .sortBy { case (res, _) => res.pValue } + .take((testResult.length * getPercentile).toInt) + case "fpr" => + testResult + .filter { case (res, _) => res.pValue < getFpr } + case "fdr" => + // This uses the Benjamini-Hochberg procedure. + // https://en.wikipedia.org/wiki/False_discovery_rate#Benjamini.E2.80.93Hochberg_procedure + val tempRes = testResult + .sortBy { case (res, _) => res.pValue } + val selected = tempRes + .zipWithIndex + .filter { case ((res, _), index) => + res.pValue <= getFdr * (index + 1) / testResult.length } + if (selected.isEmpty) { + Array.empty[(SelectionTestResult, Int)] + } else { + val maxIndex = selected.map(_._2).max + tempRes.take(maxIndex + 1) + } + case "fwe" => + testResult + .filter { case (res, _) => res.pValue < getFwe / testResult.length } + case errorType => + throw new IllegalStateException(s"Unknown Selector Type: $errorType") + } + val indices = features.map { case (_, index) => index } + copyValues(new ANOVASelectorModel(uid, indices.sorted) + .setParent(this)) + } + + @Since("3.1.0") + override def transformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) + SchemaUtils.checkNumericType(schema, $(labelCol)) + SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) + } + + @Since("3.1.0") + override def copy(extra: ParamMap): ANOVASelector = defaultCopy(extra) +} + +@Since("3.1.0") +object ANOVASelector extends DefaultParamsReadable[ANOVASelector] { + + @Since("3.1.0") + override def load(path: String): ANOVASelector = super.load(path) +} + +/** + * Model fitted by [[ANOVASelector]]. + */ +@Since("3.1.0") +class ANOVASelectorModel private[ml]( + @Since("3.1.0") override val uid: String, + @Since("3.1.0") val selectedFeatures: Array[Int]) + extends Model[ANOVASelectorModel] with FValueSelectorParams with MLWritable { + + if (selectedFeatures.length >= 2) { + require(selectedFeatures.sliding(2).forall(l => l(0) < l(1)), + "Index should be strictly increasing.") + } + + /** @group setParam */ + @Since("3.1.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("3.1.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + + @Since("3.1.0") + override def transform(dataset: Dataset[_]): DataFrame = { + val outputSchema = transformSchema(dataset.schema, logging = true) + + val newSize = selectedFeatures.length + val func = { vector: Vector => + vector match { + case SparseVector(_, indices, values) => + val (newIndices, newValues) = compressSparse(indices, values) + Vectors.sparse(newSize, newIndices, newValues) + case DenseVector(values) => + Vectors.dense(selectedFeatures.map(values)) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } + + val transformer = udf(func) + dataset.withColumn($(outputCol), transformer(col($(featuresCol))), + outputSchema($(outputCol)).metadata) + } + + @Since("3.1.0") + override def transformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) + val newField = prepOutputField(schema) + SchemaUtils.appendColumn(schema, newField) + } + + /** + * Prepare the output column field, including per-feature metadata. + */ + private def prepOutputField(schema: StructType): StructField = { + val selector = selectedFeatures.toSet + val origAttrGroup = AttributeGroup.fromStructField(schema($(featuresCol))) + val featureAttributes: Array[Attribute] = if (origAttrGroup.attributes.nonEmpty) { + origAttrGroup.attributes.get.zipWithIndex.filter(x => selector.contains(x._2)).map(_._1) + } else { + Array.fill[Attribute](selector.size)(NominalAttribute.defaultAttr) + } + val newAttributeGroup = new AttributeGroup($(outputCol), featureAttributes) + newAttributeGroup.toStructField() + } + + @Since("3.1.0") + override def copy(extra: ParamMap): ANOVASelectorModel = { + val copied = new ANOVASelectorModel(uid, selectedFeatures) + .setParent(parent) + copyValues(copied, extra) + } + + @Since("3.1.0") + override def write: MLWriter = new ANOVASelectorModel.ANOVASelectorModelWriter(this) + + @Since("3.1.0") + override def toString: String = { + s"ANOVASelectorModel: uid=$uid, numSelectedFeatures=${selectedFeatures.length}" + } + + private[spark] def compressSparse( + indices: Array[Int], + values: Array[Double]): (Array[Int], Array[Double]) = { + val newValues = new ArrayBuilder.ofDouble + val newIndices = new ArrayBuilder.ofInt + var i = 0 + var j = 0 + while (i < indices.length && j < selectedFeatures.length) { + val indicesIdx = indices(i) + val filterIndicesIdx = selectedFeatures(j) + if (indicesIdx == filterIndicesIdx) { + newIndices += j + newValues += values(i) + j += 1 + i += 1 + } else { + if (indicesIdx > filterIndicesIdx) { + j += 1 + } else { + i += 1 + } + } + } + // TODO: Sparse representation might be ineffective if (newSize ~= newValues.size) + (newIndices.result(), newValues.result()) + } +} + +@Since("3.1.0") +object ANOVASelectorModel extends MLReadable[ANOVASelectorModel] { + + @Since("3.1.0") + override def read: MLReader[ANOVASelectorModel] = + new ANOVASelectorModelReader + + @Since("3.1.0") + override def load(path: String): ANOVASelectorModel = super.load(path) + + private[ANOVASelectorModel] class ANOVASelectorModelWriter( + instance: ANOVASelectorModel) extends MLWriter { + + private case class Data(selectedFeatures: Seq[Int]) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.selectedFeatures.toSeq) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class ANOVASelectorModelReader extends + MLReader[ANOVASelectorModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[ANOVASelectorModel].getName + + override def load(path: String): ANOVASelectorModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + .select("selectedFeatures").head() + val selectedFeatures = data.getAs[Seq[Int]](0).toArray + val model = new ANOVASelectorModel(metadata.uid, selectedFeatures) + metadata.getAndSetParams(model) + model + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index f82c03206580e..40bc36c075c20 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -27,11 +27,11 @@ import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.stat.{ChiSquareTest, SelectionTestResult} +import org.apache.spark.ml.stat.ChiSquareTest import org.apache.spark.ml.util._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, StructField, StructType} +import org.apache.spark.sql.types.{StructField, StructType} /** * Params for [[ChiSqSelector]] and [[ChiSqSelectorModel]]. @@ -197,49 +197,54 @@ final class ChiSqSelector @Since("1.6.0") (@Since("1.6.0") override val uid: Str @Since("2.0.0") override def fit(dataset: Dataset[_]): ChiSqSelectorModel = { transformSchema(dataset.schema, logging = true) - dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => - LabeledPoint(label, features) + val spark = dataset.sparkSession + import spark.implicits._ + + val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) + val resultDF = ChiSquareTest.test(dataset.toDF, $(featuresCol), $(labelCol), true) + + def getTopIndices(k: Int): Array[Int] = { + resultDF.sort("pValue", "featureIndex") + .select("featureIndex") + .limit(k) + .as[Int] + .collect() } - val testResult = ChiSquareTest.testChiSquare(dataset, getFeaturesCol, getLabelCol) - .zipWithIndex - val features = $(selectorType) match { + val indices = $(selectorType) match { case "numTopFeatures" => - testResult - .sortBy { case (res, _) => res.pValue } - .take(getNumTopFeatures) + getTopIndices($(numTopFeatures)) case "percentile" => - testResult - .sortBy { case (res, _) => res.pValue } - .take((testResult.length * getPercentile).toInt) + getTopIndices((numFeatures * getPercentile).toInt) case "fpr" => - testResult - .filter { case (res, _) => res.pValue < getFpr } + resultDF.select("featureIndex") + .where(col("pValue") < $(fpr)) + .as[Int].collect() case "fdr" => // This uses the Benjamini-Hochberg procedure. // https://en.wikipedia.org/wiki/False_discovery_rate#Benjamini.E2.80.93Hochberg_procedure - val tempRes = testResult - .sortBy { case (res, _) => res.pValue } - val selected = tempRes + val f = $(fdr) / numFeatures + val maxIndex = resultDF.sort("pValue", "featureIndex") + .select("pValue") + .as[Double].rdd .zipWithIndex - .filter { case ((res, _), index) => - res.pValue <= getFdr * (index + 1) / testResult.length } - if (selected.isEmpty) { - Array.empty[(SelectionTestResult, Int)] - } else { - val maxIndex = selected.map(_._2).max - tempRes.take(maxIndex + 1) - } + .flatMap { case (pValue, index) => + if (pValue <= f * (index + 1)) { + Iterator.single(index.toInt) + } else Iterator.empty + }.fold(-1)(math.max) + if (maxIndex >= 0) { + getTopIndices(maxIndex + 1) + } else Array.emptyIntArray case "fwe" => - testResult - .filter { case (res, _) => res.pValue < getFwe / testResult.length } + resultDF.select("featureIndex") + .where(col("pValue") < $(fwe) / numFeatures) + .as[Int].collect() case errorType => throw new IllegalStateException(s"Unknown Selector Type: $errorType") } - val indices = features.map { case (_, index) => index } - copyValues(new ChiSqSelectorModel(uid, indices.sorted) - .setParent(this)) + + copyValues(new ChiSqSelectorModel(uid, indices.sorted).setParent(this)) } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/FValueSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/FValueSelector.scala index f432531f7c1df..7019590d7ae67 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/FValueSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/FValueSelector.scala @@ -31,7 +31,7 @@ import org.apache.spark.ml.stat.{FValueTest, SelectionTestResult} import org.apache.spark.ml.util._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, StructField, StructType} +import org.apache.spark.sql.types.{StructField, StructType} /** @@ -200,10 +200,6 @@ final class FValueSelector @Since("3.1.0") (override val uid: String) @Since("3.1.0") override def fit(dataset: Dataset[_]): FValueSelectorModel = { transformSchema(dataset.schema, logging = true) - dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => - LabeledPoint(label, features) - } val testResult = FValueTest.testRegression(dataset, getFeaturesCol, getLabelCol) .zipWithIndex @@ -227,7 +223,8 @@ final class FValueSelector @Since("3.1.0") (override val uid: String) val selected = tempRes .zipWithIndex .filter { case ((res, _), index) => - res.pValue <= getFdr * (index + 1) / testResult.length } + res.pValue <= getFdr * (index + 1) / testResult.length + } if (selected.isEmpty) { Array.empty[(SelectionTestResult, Int)] } else { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VarianceThresholdSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VarianceThresholdSelector.scala new file mode 100644 index 0000000000000..3eac29c2a2e2a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VarianceThresholdSelector.scala @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import scala.collection.mutable.ArrayBuilder + +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since +import org.apache.spark.ml._ +import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NominalAttribute} +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.stat.Summarizer +import org.apache.spark.ml.util._ +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{StructField, StructType} + + +/** + * Params for [[VarianceThresholdSelector]] and [[VarianceThresholdSelectorModel]]. + */ +private[feature] trait VarianceThresholdSelectorParams extends Params + with HasFeaturesCol with HasOutputCol { + + /** + * Param for variance threshold. Features with a variance not greater than this threshold + * will be removed. The default value is 0.0. + * + * @group param + */ + @Since("3.1.0") + final val varianceThreshold = new DoubleParam(this, "varianceThreshold", + "Param for variance threshold. Features with a variance not greater than this threshold" + + " will be removed. The default value is 0.0.", ParamValidators.gtEq(0)) + setDefault(varianceThreshold -> 0.0) + + /** @group getParam */ + @Since("3.1.0") + def getVarianceThreshold: Double = $(varianceThreshold) +} + +/** + * Feature selector that removes all low-variance features. Features with a + * variance not greater than the threshold will be removed. The default is to keep + * all features with non-zero variance, i.e. remove the features that have the + * same value in all samples. + */ +@Since("3.1.0") +final class VarianceThresholdSelector @Since("3.1.0")(@Since("3.1.0") override val uid: String) + extends Estimator[VarianceThresholdSelectorModel] with VarianceThresholdSelectorParams +with DefaultParamsWritable { + + @Since("3.1.0") + def this() = this(Identifiable.randomUID("VarianceThresholdSelector")) + + /** @group setParam */ + @Since("3.1.0") + def setVarianceThreshold(value: Double): this.type = set(varianceThreshold, value) + + /** @group setParam */ + @Since("3.1.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("3.1.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + + @Since("3.1.0") + override def fit(dataset: Dataset[_]): VarianceThresholdSelectorModel = { + transformSchema(dataset.schema, logging = true) + + val Row(maxs: Vector, mins: Vector, variances: Vector) = dataset + .select(Summarizer.metrics("max", "min", "variance").summary(col($(featuresCol))) + .as("summary")) + .select("summary.max", "summary.min", "summary.variance") + .first() + + val numFeatures = maxs.size + val indices = Array.tabulate(numFeatures) { i => + // Use peak-to-peak to avoid numeric precision issues for constant features + (i, if (maxs(i) == mins(i)) 0.0 else variances(i)) + }.filter(_._2 > getVarianceThreshold).map(_._1) + copyValues(new VarianceThresholdSelectorModel(uid, indices.sorted) + .setParent(this)) + } + + @Since("3.1.0") + override def transformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) + SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) + } + + @Since("3.1.0") + override def copy(extra: ParamMap): VarianceThresholdSelector = defaultCopy(extra) +} + +@Since("3.1.0") +object VarianceThresholdSelector extends DefaultParamsReadable[VarianceThresholdSelector] { + + @Since("3.1.0") + override def load(path: String): VarianceThresholdSelector = super.load(path) +} + +/** + * Model fitted by [[VarianceThresholdSelector]]. + */ +@Since("3.1.0") +class VarianceThresholdSelectorModel private[ml]( + @Since("3.1.0") override val uid: String, + @Since("3.1.0") val selectedFeatures: Array[Int]) + extends Model[VarianceThresholdSelectorModel] with VarianceThresholdSelectorParams + with MLWritable { + + if (selectedFeatures.length >= 2) { + require(selectedFeatures.sliding(2).forall(l => l(0) < l(1)), + "Index should be strictly increasing.") + } + + /** @group setParam */ + @Since("3.1.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("3.1.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + + @Since("3.1.0") + override def transform(dataset: Dataset[_]): DataFrame = { + val outputSchema = transformSchema(dataset.schema, logging = true) + + val newSize = selectedFeatures.length + val func = { vector: Vector => + vector match { + case SparseVector(_, indices, values) => + val (newIndices, newValues) = compressSparse(indices, values) + Vectors.sparse(newSize, newIndices, newValues) + case DenseVector(values) => + Vectors.dense(selectedFeatures.map(values)) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } + + val transformer = udf(func) + dataset.withColumn($(outputCol), transformer(col($(featuresCol))), + outputSchema($(outputCol)).metadata) + } + + @Since("3.1.0") + override def transformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) + val newField = prepOutputField(schema) + SchemaUtils.appendColumn(schema, newField) + } + + /** + * Prepare the output column field, including per-feature metadata. + */ + private def prepOutputField(schema: StructType): StructField = { + val selector = selectedFeatures.toSet + val origAttrGroup = AttributeGroup.fromStructField(schema($(featuresCol))) + val featureAttributes: Array[Attribute] = if (origAttrGroup.attributes.nonEmpty) { + origAttrGroup.attributes.get.zipWithIndex.filter(x => selector.contains(x._2)).map(_._1) + } else { + Array.fill[Attribute](selector.size)(NominalAttribute.defaultAttr) + } + val newAttributeGroup = new AttributeGroup($(outputCol), featureAttributes) + newAttributeGroup.toStructField() + } + + @Since("3.1.0") + override def copy(extra: ParamMap): VarianceThresholdSelectorModel = { + val copied = new VarianceThresholdSelectorModel(uid, selectedFeatures) + .setParent(parent) + copyValues(copied, extra) + } + + @Since("3.1.0") + override def write: MLWriter = + new VarianceThresholdSelectorModel.VarianceThresholdSelectorWriter(this) + + @Since("3.1.0") + override def toString: String = { + s"VarianceThresholdSelectorModel: uid=$uid, numSelectedFeatures=${selectedFeatures.length}" + } + + private[spark] def compressSparse( + indices: Array[Int], + values: Array[Double]): (Array[Int], Array[Double]) = { + val newValues = new ArrayBuilder.ofDouble + val newIndices = new ArrayBuilder.ofInt + var i = 0 + var j = 0 + while (i < indices.length && j < selectedFeatures.length) { + val indicesIdx = indices(i) + val filterIndicesIdx = selectedFeatures(j) + if (indicesIdx == filterIndicesIdx) { + newIndices += j + newValues += values(i) + j += 1 + i += 1 + } else { + if (indicesIdx > filterIndicesIdx) { + j += 1 + } else { + i += 1 + } + } + } + // TODO: Sparse representation might be ineffective if (newSize ~= newValues.size) + (newIndices.result(), newValues.result()) + } +} + +@Since("3.1.0") +object VarianceThresholdSelectorModel extends MLReadable[VarianceThresholdSelectorModel] { + + @Since("3.1.0") + override def read: MLReader[VarianceThresholdSelectorModel] = + new VarianceThresholdSelectorModelReader + + @Since("3.1.0") + override def load(path: String): VarianceThresholdSelectorModel = super.load(path) + + private[VarianceThresholdSelectorModel] class VarianceThresholdSelectorWriter( + instance: VarianceThresholdSelectorModel) extends MLWriter { + + private case class Data(selectedFeatures: Seq[Int]) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.selectedFeatures.toSeq) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class VarianceThresholdSelectorModelReader extends + MLReader[VarianceThresholdSelectorModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[VarianceThresholdSelectorModel].getName + + override def load(path: String): VarianceThresholdSelectorModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + .select("selectedFeatures").head() + val selectedFeatures = data.getAs[Seq[Int]](0).toArray + val model = new VarianceThresholdSelectorModel(metadata.uid, selectedFeatures) + metadata.getAndSetParams(model) + model + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala index a66ba27a7b9c5..1cab37a25ad57 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala @@ -17,15 +17,13 @@ package org.apache.spark.ml.linalg -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.sql.types.DataType /** - * :: DeveloperApi :: * SQL data types for vectors and matrices. */ @Since("2.0.0") -@DeveloperApi object SQLDataTypes { /** Data type for [[Vector]]. */ 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 4c50f1e3292bc..53ca35ccd0073 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 @@ -29,12 +29,11 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkException -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.{JsonMatrixConverter, JsonVectorConverter, Matrix, Vector} import org.apache.spark.ml.util.Identifiable /** - * :: DeveloperApi :: * A param with self-contained documentation and optionally default value. Primitive-typed param * should use the specialized versions, which are more friendly to Java users. * @@ -45,7 +44,6 @@ import org.apache.spark.ml.util.Identifiable * See [[ParamValidators]] for factory methods for common validation functions. * @tparam T param value type */ -@DeveloperApi class Param[T](val parent: String, val name: String, val doc: String, val isValid: T => Boolean) extends Serializable { @@ -159,11 +157,9 @@ private[ml] object Param { } /** - * :: DeveloperApi :: * Factory methods for common validation functions for `Param.isValid`. * The numerical methods only support Int, Long, Float, and Double. */ -@DeveloperApi object ParamValidators { /** (private[param]) Default validation always return true */ @@ -323,10 +319,8 @@ object ParamValidators { // specialize primitive-typed params because Java doesn't recognize scala.Double, scala.Int, ... /** - * :: DeveloperApi :: * Specialized version of `Param[Double]` for Java. */ -@DeveloperApi class DoubleParam(parent: String, name: String, doc: String, isValid: Double => Boolean) extends Param[Double](parent, name, doc, isValid) { @@ -383,10 +377,8 @@ private[param] object DoubleParam { } /** - * :: DeveloperApi :: * Specialized version of `Param[Int]` for Java. */ -@DeveloperApi class IntParam(parent: String, name: String, doc: String, isValid: Int => Boolean) extends Param[Int](parent, name, doc, isValid) { @@ -412,10 +404,8 @@ class IntParam(parent: String, name: String, doc: String, isValid: Int => Boolea } /** - * :: DeveloperApi :: * Specialized version of `Param[Float]` for Java. */ -@DeveloperApi class FloatParam(parent: String, name: String, doc: String, isValid: Float => Boolean) extends Param[Float](parent, name, doc, isValid) { @@ -473,10 +463,8 @@ private object FloatParam { } /** - * :: DeveloperApi :: * Specialized version of `Param[Long]` for Java. */ -@DeveloperApi class LongParam(parent: String, name: String, doc: String, isValid: Long => Boolean) extends Param[Long](parent, name, doc, isValid) { @@ -502,10 +490,8 @@ class LongParam(parent: String, name: String, doc: String, isValid: Long => Bool } /** - * :: DeveloperApi :: * Specialized version of `Param[Boolean]` for Java. */ -@DeveloperApi class BooleanParam(parent: String, name: String, doc: String) // No need for isValid extends Param[Boolean](parent, name, doc) { @@ -525,10 +511,8 @@ class BooleanParam(parent: String, name: String, doc: String) // No need for isV } /** - * :: DeveloperApi :: * Specialized version of `Param[Array[String]]` for Java. */ -@DeveloperApi class StringArrayParam(parent: Params, name: String, doc: String, isValid: Array[String] => Boolean) extends Param[Array[String]](parent, name, doc, isValid) { @@ -550,10 +534,8 @@ class StringArrayParam(parent: Params, name: String, doc: String, isValid: Array } /** - * :: DeveloperApi :: * Specialized version of `Param[Array[Double]]` for Java. */ -@DeveloperApi class DoubleArrayParam(parent: Params, name: String, doc: String, isValid: Array[Double] => Boolean) extends Param[Array[Double]](parent, name, doc, isValid) { @@ -580,10 +562,8 @@ class DoubleArrayParam(parent: Params, name: String, doc: String, isValid: Array } /** - * :: DeveloperApi :: * Specialized version of `Param[Array[Array[Double]]]` for Java. */ -@DeveloperApi class DoubleArrayArrayParam( parent: Params, name: String, @@ -619,10 +599,8 @@ class DoubleArrayArrayParam( } /** - * :: DeveloperApi :: * Specialized version of `Param[Array[Int]]` for Java. */ -@DeveloperApi class IntArrayParam(parent: Params, name: String, doc: String, isValid: Array[Int] => Boolean) extends Param[Array[Int]](parent, name, doc, isValid) { @@ -657,11 +635,9 @@ case class ParamPair[T] @Since("1.2.0") ( } /** - * :: DeveloperApi :: * Trait for components that take parameters. This also provides an internal param map to store * parameter values attached to the instance. */ -@DeveloperApi trait Params extends Identifiable with Serializable { /** @@ -915,13 +891,11 @@ private[ml] object Params { } /** - * :: DeveloperApi :: * Java-friendly wrapper for [[Params]]. * Java developers who need to extend [[Params]] should use this class instead. * If you need to extend an abstract class which already extends [[Params]], then that abstract * class should be Java-friendly as well. */ -@DeveloperApi abstract class JavaParams extends Params /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index 6194dfacf01fc..7fd5f5938b565 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -207,7 +207,6 @@ private[shared] object SharedParamsCodeGen { | * Trait for shared param $name$defaultValueDoc. This trait may be changed or | * removed between minor versions. | */ - |@DeveloperApi |trait Has$Name extends Params { | | /** @@ -244,7 +243,6 @@ private[shared] object SharedParamsCodeGen { | |package org.apache.spark.ml.param.shared | - |import org.apache.spark.annotation.DeveloperApi |import org.apache.spark.ml.param._ | |// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 0c0d2b5981dbe..60203eba61ea5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -17,7 +17,6 @@ package org.apache.spark.ml.param.shared -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.param._ // DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. @@ -28,7 +27,6 @@ import org.apache.spark.ml.param._ * Trait for shared param regParam. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasRegParam extends Params { /** @@ -45,7 +43,6 @@ trait HasRegParam extends Params { * Trait for shared param maxIter. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasMaxIter extends Params { /** @@ -62,7 +59,6 @@ trait HasMaxIter extends Params { * Trait for shared param featuresCol (default: "features"). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasFeaturesCol extends Params { /** @@ -81,7 +77,6 @@ trait HasFeaturesCol extends Params { * Trait for shared param labelCol (default: "label"). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasLabelCol extends Params { /** @@ -100,7 +95,6 @@ trait HasLabelCol extends Params { * Trait for shared param predictionCol (default: "prediction"). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasPredictionCol extends Params { /** @@ -119,7 +113,6 @@ trait HasPredictionCol extends Params { * Trait for shared param rawPredictionCol (default: "rawPrediction"). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasRawPredictionCol extends Params { /** @@ -138,7 +131,6 @@ trait HasRawPredictionCol extends Params { * Trait for shared param probabilityCol (default: "probability"). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasProbabilityCol extends Params { /** @@ -157,7 +149,6 @@ trait HasProbabilityCol extends Params { * Trait for shared param varianceCol. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasVarianceCol extends Params { /** @@ -174,7 +165,6 @@ trait HasVarianceCol extends Params { * Trait for shared param threshold. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasThreshold extends Params { /** @@ -191,7 +181,6 @@ trait HasThreshold extends Params { * Trait for shared param thresholds. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasThresholds extends Params { /** @@ -208,7 +197,6 @@ trait HasThresholds extends Params { * Trait for shared param inputCol. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasInputCol extends Params { /** @@ -225,7 +213,6 @@ trait HasInputCol extends Params { * Trait for shared param inputCols. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasInputCols extends Params { /** @@ -242,7 +229,6 @@ trait HasInputCols extends Params { * Trait for shared param outputCol (default: uid + "__output"). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasOutputCol extends Params { /** @@ -261,7 +247,6 @@ trait HasOutputCol extends Params { * Trait for shared param outputCols. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasOutputCols extends Params { /** @@ -278,7 +263,6 @@ trait HasOutputCols extends Params { * Trait for shared param numFeatures (default: 262144). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasNumFeatures extends Params { /** @@ -297,7 +281,6 @@ trait HasNumFeatures extends Params { * Trait for shared param checkpointInterval. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasCheckpointInterval extends Params { /** @@ -314,7 +297,6 @@ trait HasCheckpointInterval extends Params { * Trait for shared param fitIntercept (default: true). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasFitIntercept extends Params { /** @@ -333,7 +315,6 @@ trait HasFitIntercept extends Params { * Trait for shared param handleInvalid. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasHandleInvalid extends Params { /** @@ -350,7 +331,6 @@ trait HasHandleInvalid extends Params { * Trait for shared param standardization (default: true). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasStandardization extends Params { /** @@ -369,7 +349,6 @@ trait HasStandardization extends Params { * Trait for shared param seed (default: this.getClass.getName.hashCode.toLong). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasSeed extends Params { /** @@ -388,7 +367,6 @@ trait HasSeed extends Params { * Trait for shared param elasticNetParam. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasElasticNetParam extends Params { /** @@ -405,7 +383,6 @@ trait HasElasticNetParam extends Params { * Trait for shared param tol. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasTol extends Params { /** @@ -422,7 +399,6 @@ trait HasTol extends Params { * Trait for shared param relativeError (default: 0.001). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasRelativeError extends Params { /** @@ -441,7 +417,6 @@ trait HasRelativeError extends Params { * Trait for shared param stepSize. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasStepSize extends Params { /** @@ -458,7 +433,6 @@ trait HasStepSize extends Params { * Trait for shared param weightCol. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasWeightCol extends Params { /** @@ -475,7 +449,6 @@ trait HasWeightCol extends Params { * Trait for shared param solver. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasSolver extends Params { /** @@ -492,7 +465,6 @@ trait HasSolver extends Params { * Trait for shared param aggregationDepth (default: 2). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasAggregationDepth extends Params { /** @@ -511,7 +483,6 @@ trait HasAggregationDepth extends Params { * Trait for shared param collectSubModels (default: false). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasCollectSubModels extends Params { /** @@ -530,7 +501,6 @@ trait HasCollectSubModels extends Params { * Trait for shared param loss. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasLoss extends Params { /** @@ -547,7 +517,6 @@ trait HasLoss extends Params { * Trait for shared param distanceMeasure (default: "euclidean"). This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasDistanceMeasure extends Params { /** @@ -566,7 +535,6 @@ trait HasDistanceMeasure extends Params { * Trait for shared param validationIndicatorCol. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasValidationIndicatorCol extends Params { /** @@ -583,7 +551,6 @@ trait HasValidationIndicatorCol extends Params { * Trait for shared param blockSize. This trait may be changed or * removed between minor versions. */ -@DeveloperApi trait HasBlockSize extends Params { /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassificationWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala similarity index 100% rename from mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassificationWrapper.scala rename to mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala similarity index 100% rename from mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressionWrapper.scala rename to mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala new file mode 100644 index 0000000000000..a6c6ad6881a02 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.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.classification.{FMClassificationModel, FMClassifier} +import org.apache.spark.ml.feature.{IndexToString, RFormula} +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class FMClassifierWrapper private ( + val pipeline: PipelineModel, + val features: Array[String], + val labels: Array[String]) extends MLWritable { + import FMClassifierWrapper._ + + private val fmClassificationModel: FMClassificationModel = + pipeline.stages(1).asInstanceOf[FMClassificationModel] + + lazy val rFeatures: Array[String] = if (fmClassificationModel.getFitIntercept) { + Array("(Intercept)") ++ features + } else { + features + } + + lazy val rCoefficients: Array[Double] = if (fmClassificationModel.getFitIntercept) { + Array(fmClassificationModel.intercept) ++ fmClassificationModel.linear.toArray + } else { + fmClassificationModel.linear.toArray + } + + lazy val rFactors = fmClassificationModel.factors.toArray + + lazy val numClasses: Int = fmClassificationModel.numClasses + + lazy val numFeatures: Int = fmClassificationModel.numFeatures + + lazy val factorSize: Int = fmClassificationModel.getFactorSize + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(PREDICTED_LABEL_INDEX_COL) + .drop(fmClassificationModel.getFeaturesCol) + .drop(fmClassificationModel.getLabelCol) + } + + override def write: MLWriter = new FMClassifierWrapper.FMClassifierWrapperWriter(this) +} + +private[r] object FMClassifierWrapper + extends MLReadable[FMClassifierWrapper] { + + val PREDICTED_LABEL_INDEX_COL = "pred_label_idx" + val PREDICTED_LABEL_COL = "prediction" + + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + factorSize: Int, + fitLinear: Boolean, + regParam: Double, + miniBatchFraction: Double, + initStd: Double, + maxIter: Int, + stepSize: Double, + tol: Double, + solver: String, + seed: String, + thresholds: Array[Double], + handleInvalid: String): FMClassifierWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setForceIndexLabel(true) + .setHandleInvalid(handleInvalid) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + val fitIntercept = rFormula.hasIntercept + + // get labels and feature names from output schema + val (features, labels) = getFeaturesAndLabels(rFormulaModel, data) + + // assemble and fit the pipeline + val fmc = new FMClassifier() + .setFactorSize(factorSize) + .setFitIntercept(fitIntercept) + .setFitLinear(fitLinear) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) + .setInitStd(initStd) + .setMaxIter(maxIter) + .setStepSize(stepSize) + .setTol(tol) + .setSolver(solver) + .setFeaturesCol(rFormula.getFeaturesCol) + .setLabelCol(rFormula.getLabelCol) + .setPredictionCol(PREDICTED_LABEL_INDEX_COL) + + if (seed != null && seed.length > 0) { + fmc.setSeed(seed.toLong) + } + + if (thresholds != null) { + fmc.setThresholds(thresholds) + } + + val idxToStr = new IndexToString() + .setInputCol(PREDICTED_LABEL_INDEX_COL) + .setOutputCol(PREDICTED_LABEL_COL) + .setLabels(labels) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, fmc, idxToStr)) + .fit(data) + + new FMClassifierWrapper(pipeline, features, labels) + } + + override def read: MLReader[FMClassifierWrapper] = new FMClassifierWrapperReader + + class FMClassifierWrapperWriter(instance: FMClassifierWrapper) 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) ~ + ("labels" -> instance.labels.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class FMClassifierWrapperReader extends MLReader[FMClassifierWrapper] { + + override def load(path: String): FMClassifierWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + val labels = (rMetadata \ "labels").extract[Array[String]] + + val pipeline = PipelineModel.load(pipelinePath) + new FMClassifierWrapper(pipeline, features, labels) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala new file mode 100644 index 0000000000000..ae2cf68781b77 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala @@ -0,0 +1,155 @@ +/* + * 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.feature.RFormula +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.regression.{FMRegressionModel, FMRegressor} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class FMRegressorWrapper private ( + val pipeline: PipelineModel, + val features: Array[String]) extends MLWritable { + + private val fmRegressionModel: FMRegressionModel = + pipeline.stages(1).asInstanceOf[FMRegressionModel] + + lazy val rFeatures: Array[String] = if (fmRegressionModel.getFitIntercept) { + Array("(Intercept)") ++ features + } else { + features + } + + lazy val rCoefficients: Array[Double] = if (fmRegressionModel.getFitIntercept) { + Array(fmRegressionModel.intercept) ++ fmRegressionModel.linear.toArray + } else { + fmRegressionModel.linear.toArray + } + + lazy val rFactors = fmRegressionModel.factors.toArray + + lazy val numFeatures: Int = fmRegressionModel.numFeatures + + lazy val factorSize: Int = fmRegressionModel.getFactorSize + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(fmRegressionModel.getFeaturesCol) + } + + override def write: MLWriter = new FMRegressorWrapper.FMRegressorWrapperWriter(this) +} + +private[r] object FMRegressorWrapper + extends MLReadable[FMRegressorWrapper] { + + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + factorSize: Int, + fitLinear: Boolean, + regParam: Double, + miniBatchFraction: Double, + initStd: Double, + maxIter: Int, + stepSize: Double, + tol: Double, + solver: String, + seed: String, + stringIndexerOrderType: String): FMRegressorWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setStringIndexerOrderType(stringIndexerOrderType) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + val fitIntercept = rFormula.hasIntercept + + // 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) + + // assemble and fit the pipeline + val fmr = new FMRegressor() + .setFactorSize(factorSize) + .setFitIntercept(fitIntercept) + .setFitLinear(fitLinear) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) + .setInitStd(initStd) + .setMaxIter(maxIter) + .setStepSize(stepSize) + .setTol(tol) + .setSolver(solver) + .setFeaturesCol(rFormula.getFeaturesCol) + + if (seed != null && seed.length > 0) { + fmr.setSeed(seed.toLong) + } + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, fmr)) + .fit(data) + + new FMRegressorWrapper(pipeline, features) + } + + override def read: MLReader[FMRegressorWrapper] = new FMRegressorWrapperReader + + class FMRegressorWrapperWriter(instance: FMRegressorWrapper) 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) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class FMRegressorWrapperReader extends MLReader[FMRegressorWrapper] { + + override def load(path: String): FMRegressorWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + + val pipeline = PipelineModel.load(pipelinePath) + new FMRegressorWrapper(pipeline, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassificationWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala similarity index 100% rename from mllib/src/main/scala/org/apache/spark/ml/r/GBTClassificationWrapper.scala rename to mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala similarity index 100% rename from mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressionWrapper.scala rename to mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala new file mode 100644 index 0000000000000..19cd4dff0acee --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.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.feature.RFormula +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.regression.{LinearRegression, LinearRegressionModel} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class LinearRegressionWrapper private ( + val pipeline: PipelineModel, + val features: Array[String]) extends MLWritable { + + private val linearRegressionModel: LinearRegressionModel = + pipeline.stages(1).asInstanceOf[LinearRegressionModel] + + lazy val rFeatures: Array[String] = if (linearRegressionModel.getFitIntercept) { + Array("(Intercept)") ++ features + } else { + features + } + + lazy val rCoefficients: Array[Double] = if (linearRegressionModel.getFitIntercept) { + Array(linearRegressionModel.intercept) ++ linearRegressionModel.coefficients.toArray + } else { + linearRegressionModel.coefficients.toArray + } + + lazy val numFeatures: Int = linearRegressionModel.numFeatures + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(linearRegressionModel.getFeaturesCol) + } + + override def write: MLWriter = new LinearRegressionWrapper.LinearRegressionWrapperWriter(this) +} + +private[r] object LinearRegressionWrapper + extends MLReadable[LinearRegressionWrapper] { + + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + maxIter: Int, + regParam: Double, + elasticNetParam: Double, + tol: Double, + standardization: Boolean, + solver: String, + weightCol: String, + aggregationDepth: Int, + loss: String, + epsilon: Double, + stringIndexerOrderType: String): LinearRegressionWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setStringIndexerOrderType(stringIndexerOrderType) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + val fitIntercept = rFormula.hasIntercept + + // 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) + + // assemble and fit the pipeline + val lm = new LinearRegression() + .setMaxIter(maxIter) + .setRegParam(regParam) + .setElasticNetParam(elasticNetParam) + .setTol(tol) + .setFitIntercept(fitIntercept) + .setStandardization(standardization) + .setSolver(solver) + .setAggregationDepth(aggregationDepth) + .setLoss(loss) + .setEpsilon(epsilon) + .setFeaturesCol(rFormula.getFeaturesCol) + + if (weightCol != null) { + lm.setWeightCol(weightCol) + } + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, lm)) + .fit(data) + + new LinearRegressionWrapper(pipeline, features) + } + + override def read: MLReader[LinearRegressionWrapper] = new LinearRegressionWrapperReader + + class LinearRegressionWrapperWriter(instance: LinearRegressionWrapper) 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) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class LinearRegressionWrapperReader extends MLReader[LinearRegressionWrapper] { + + override def load(path: String): LinearRegressionWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + + val pipeline = PipelineModel.load(pipelinePath) + new LinearRegressionWrapper(pipeline, features) + } + } +} 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 ba6445a730306..551c7514ee85f 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 @@ -74,6 +74,12 @@ private[r] object RWrappers extends MLReader[Object] { LinearSVCWrapper.load(path) case "org.apache.spark.ml.r.FPGrowthWrapper" => FPGrowthWrapper.load(path) + case "org.apache.spark.ml.r.FMClassifierWrapper" => + FMClassifierWrapper.load(path) + case "org.apache.spark.ml.r.LinearRegressionWrapper" => + LinearRegressionWrapper.load(path) + case "org.apache.spark.ml.r.FMRegressorWrapper" => + FMRegressorWrapper.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/r/RandomForestClassificationWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala similarity index 100% rename from mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala rename to mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala similarity index 100% rename from mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala rename to mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala 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 9404dcdb6f2d4..a0e5924a7ee3a 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 @@ -32,7 +32,7 @@ import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext, SparkException} -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.BLAS @@ -731,21 +731,17 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] /** - * :: DeveloperApi :: * An implementation of ALS that supports generic ID types, specialized for Int and Long. This is * exposed as a developer API for users who do need other ID types. But it is not recommended * because it increases the shuffle size and memory requirement during training. For simplicity, * users and items must have the same type. The number of distinct users/items should be smaller * than 2 billion. */ -@DeveloperApi object ALS extends DefaultParamsReadable[ALS] with Logging { /** - * :: DeveloperApi :: * Rating class for better code readability. */ - @DeveloperApi case class Rating[@specialized(Int, Long) ID](user: ID, item: ID, rating: Float) @Since("1.6.0") @@ -908,7 +904,6 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { } /** - * :: DeveloperApi :: * Implementation of the ALS algorithm. * * This implementation of the ALS factorization algorithm partitions the two sets of factors among @@ -933,7 +928,6 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { * "block" as referring to a subset of an RDD containing the ratings rather than a contiguous * submatrix of the ratings matrix. */ - @DeveloperApi def train[ID: ClassTag]( // scalastyle:ignore ratings: RDD[Rating[ID]], rank: Int = 10, @@ -1009,7 +1003,6 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { previousItemFactors.unpersist() itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) // TODO: Generalize PeriodicGraphCheckpointer and use it here. - val deps = itemFactors.dependencies if (shouldCheckpoint(iter)) { itemFactors.checkpoint() // itemFactors gets materialized in computeFactors } @@ -1017,7 +1010,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, itemLocalIndexEncoder, implicitPrefs, alpha, solver) if (shouldCheckpoint(iter)) { - ALS.cleanShuffleDependencies(sc, deps) + itemFactors.cleanShuffleDependencies() deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -1030,10 +1023,9 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { userLocalIndexEncoder, solver = solver) if (shouldCheckpoint(iter)) { itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) - val deps = itemFactors.dependencies itemFactors.checkpoint() itemFactors.count() // checkpoint item factors and cut lineage - ALS.cleanShuffleDependencies(sc, deps) + itemFactors.cleanShuffleDependencies() deletePreviousCheckpointFile() previousCachedItemFactors.foreach(_.unpersist()) @@ -1821,31 +1813,4 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { * satisfies this requirement, we simply use a type alias here. */ private[recommendation] type ALSPartitioner = org.apache.spark.HashPartitioner - - /** - * Private function to clean up all of the shuffles files from the dependencies and their parents. - */ - private[spark] def cleanShuffleDependencies[T]( - sc: SparkContext, - deps: Seq[Dependency[_]], - blocking: Boolean = false): Unit = { - // If there is no reference tracking we skip clean up. - sc.cleaner.foreach { cleaner => - /** - * Clean the shuffles & all of its parents. - */ - def cleanEagerly(dep: Dependency[_]): Unit = { - if (dep.isInstanceOf[ShuffleDependency[_, _, _]]) { - val shuffleId = dep.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId - cleaner.doCleanupShuffle(shuffleId, blocking) - } - val rdd = dep.rdd - val rddDeps = rdd.dependencies - if (rdd.getStorageLevel == StorageLevel.NONE && rddDeps != null) { - rddDeps.foreach(cleanEagerly) - } - } - deps.foreach(cleanEagerly) - } - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index 314cf422be87e..c28dac6850e85 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -17,7 +17,6 @@ package org.apache.spark.ml.regression -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} @@ -28,7 +27,6 @@ import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} * @tparam Learner Concrete Estimator type * @tparam M Concrete Model type */ -@DeveloperApi abstract class Regressor[ FeaturesType, Learner <: Regressor[FeaturesType, Learner, M], @@ -39,14 +37,11 @@ abstract class Regressor[ } /** - * :: DeveloperApi :: - * * Model produced by a `Regressor`. * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam M Concrete Model type. */ -@DeveloperApi abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] extends PredictionModel[FeaturesType, M] with PredictorParams { diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/ANOVATest.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/ANOVATest.scala new file mode 100644 index 0000000000000..4a8f7ea9b2bf5 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/ANOVATest.scala @@ -0,0 +1,214 @@ +/* + * 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.stat + +import org.apache.commons.math3.distribution.FDistribution + +import org.apache.spark.annotation.Since +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.util.SchemaUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.util.collection.OpenHashMap + + +/** + * ANOVA Test for continuous data. + * + * See Wikipedia for more + * information on ANOVA test. + */ +@Since("3.1.0") +object ANOVATest { + + /** Used to construct output schema of tests */ + private case class ANOVAResult( + pValues: Vector, + degreesOfFreedom: Array[Long], + fValues: Vector) + + /** + * @param dataset DataFrame of categorical labels and continuous features. + * @param featuresCol Name of features column in dataset, of type `Vector` (`VectorUDT`) + * @param labelCol Name of label column in dataset, of any numerical type + * @return DataFrame containing the test result for every feature against the label. + * This DataFrame will contain a single Row with the following fields: + * - `pValues: Vector` + * - `degreesOfFreedom: Array[Long]` + * - `fValues: Vector` + * Each of these fields has one value per feature. + */ + @Since("3.1.0") + def test(dataset: DataFrame, featuresCol: String, labelCol: String): DataFrame = { + val spark = dataset.sparkSession + val testResults = testClassification(dataset, featuresCol, labelCol) + val pValues = Vectors.dense(testResults.map(_.pValue)) + val degreesOfFreedom = testResults.map(_.degreesOfFreedom) + val fValues = Vectors.dense(testResults.map(_.statistic)) + spark.createDataFrame(Seq(ANOVAResult(pValues, degreesOfFreedom, fValues))) + } + + /** + * @param dataset DataFrame of categorical labels and continuous features. + * @param featuresCol Name of features column in dataset, of type `Vector` (`VectorUDT`) + * @param labelCol Name of label column in dataset, of any numerical type + * @return Array containing the ANOVATestResult for every feature against the + * label. + */ + private[ml] def testClassification( + dataset: Dataset[_], + featuresCol: String, + labelCol: String): Array[SelectionTestResult] = { + val spark = dataset.sparkSession + import spark.implicits._ + + SchemaUtils.checkColumnType(dataset.schema, featuresCol, new VectorUDT) + SchemaUtils.checkNumericType(dataset.schema, labelCol) + + val points = dataset.select(col(labelCol).cast("double"), col(featuresCol)) + .as[(Double, Vector)].rdd + + points.first()._2 match { + case dv: DenseVector => + testClassificationDenseFeatures(points, dv.size) + case sv: SparseVector => + testClassificationSparseFeatures(points, sv.size) + } + } + + private def testClassificationDenseFeatures( + points: RDD[(Double, Vector)], + numFeatures: Int): Array[SelectionTestResult] = { + points.flatMap { case (label, features) => + require(features.size == numFeatures, + s"Number of features must be $numFeatures but got ${features.size}") + features.iterator.map { case (col, value) => (col, (label, value)) } + }.aggregateByKey[(Double, Double, OpenHashMap[Double, Double], OpenHashMap[Double, Long])]( + (0.0, 0.0, new OpenHashMap[Double, Double], new OpenHashMap[Double, Long]))( + seqOp = { + case ((sum, sumOfSq, sums, counts), (label, value)) => + // sums: mapOfSumPerClass (key: label, value: sum of features for each label) + // counts: mapOfCountPerClass key: label, value: count of features for each label + sums.changeValue(label, value, _ + value) + counts.changeValue(label, 1L, _ + 1L) + (sum + value, sumOfSq + value * value, sums, counts) + }, + combOp = { + case ((sum1, sumOfSq1, sums1, counts1), (sum2, sumOfSq2, sums2, counts2)) => + sums2.foreach { case (v, w) => sums1.changeValue(v, w, _ + w) } + counts2.foreach { case (v, w) => counts1.changeValue(v, w, _ + w) } + (sum1 + sum2, sumOfSq1 + sumOfSq2, sums1, counts1) + } + ).mapValues { case (sum, sumOfSq, sums, counts) => + computeANOVA(sum, sumOfSq, sums.toMap, counts.toMap) + }.collect().sortBy(_._1).map { + case (_, (pValue, degreesOfFreedom, fValue)) => + new ANOVATestResult(pValue, degreesOfFreedom, fValue) + } + } + + private def testClassificationSparseFeatures( + points: RDD[(Double, Vector)], + numFeatures: Int): Array[SelectionTestResult] = { + val counts = points.map(_._1).countByValue().toMap + + val numParts = points.getNumPartitions + points.mapPartitionsWithIndex { case (pid, iter) => + iter.flatMap { case (label, features) => + require(features.size == numFeatures, + s"Number of features must be $numFeatures but got ${features.size}") + features.nonZeroIterator.map { case (col, value) => (col, (label, value)) } + } ++ { + // append this to make sure that all columns are taken into account + Iterator.range(0, numFeatures) + .filter(_ % numParts == pid) + .map(col => (col, null)) + } + }.aggregateByKey[(Double, Double, OpenHashMap[Double, Double])]( + (0.0, 0.0, new OpenHashMap[Double, Double]))( + seqOp = { + case ((sum, sumOfSq, sums), labelAndValue) => + // sums: mapOfSumPerClass (key: label, value: sum of features for each label) + if (labelAndValue != null) { + val (label, value) = labelAndValue + sums.changeValue(label, value, _ + value) + (sum + value, sumOfSq + value * value, sums) + } else { + (sum, sumOfSq, sums) + } + }, + combOp = { + case ((sum1, sumOfSq1, sums1), (sum2, sumOfSq2, sums2)) => + sums2.foreach { case (v, w) => sums1.changeValue(v, w, _ + w) } + (sum1 + sum2, sumOfSq1 + sumOfSq2, sums1) + } + ).mapValues { case (sum, sumOfSq, sums) => + counts.keysIterator.foreach { label => + // adjust sums if all related feature values are 0 for some label + if (!sums.contains(label)) sums.update(label, 0.0) + } + computeANOVA(sum, sumOfSq, sums.toMap, counts) + }.collect().sortBy(_._1).map { + case (_, (pValue, degreesOfFreedom, fValue)) => + new ANOVATestResult(pValue, degreesOfFreedom, fValue) + } + } + + private def computeANOVA( + sum: Double, + sumOfSq: Double, + sums: Map[Double, Double], + counts: Map[Double, Long]): (Double, Long, Double) = { + val numSamples = counts.valuesIterator.sum + val numClasses = counts.size + + // e.g. features are [3.3, 2.5, 1.0, 3.0, 2.0] and labels are [1, 2, 1, 3, 3] + // sum: sum of all the features (3.3+2.5+1.0+3.0+2.0) + // sumOfSq: sum of squares of all the features (3.3^2+2.5^2+1.0^2+3.0^2+2.0^2) + // sums: mapOfSumPerClass (key: label, value: sum of features for each label) + // ( 1 -> 3.3 + 1.0, 2 -> 2.5, 3 -> 3.0 + 2.0 ) + // counts: mapOfCountPerClass (key: label, value: count of features for each label) + // ( 1 -> 2, 2 -> 2, 3 -> 2 ) + // sqSum: square of sum of all data ((3.3+2.5+1.0+3.0+2.0)^2) + val sqSum = sum * sum + val ssTot = sumOfSq - sqSum / numSamples + + // totalSqSum: + // sum( sq_sum_classes[k] / n_samples_per_class[k] for k in range(n_classes)) + // e.g. ((3.3+1.0)^2 / 2 + 2.5^2 / 1 + (3.0+2.0)^2 / 2) + val totalSqSum = sums.iterator + .map { case (label, sum) => sum * sum / counts(label) }.sum + // Sums of Squares Between + val ssbn = totalSqSum - (sqSum / numSamples) + // Sums of Squares Within + val sswn = ssTot - ssbn + // degrees of freedom between + val dfbn = numClasses - 1 + // degrees of freedom within + val dfwn = numSamples - numClasses + // mean square between + val msb = ssbn / dfbn + // mean square within + val msw = sswn / dfwn + val fValue = msb / msw + val pValue = 1 - new FDistribution(dfbn, dfwn).cumulativeProbability(fValue) + val degreesOfFreedom = dfbn + dfwn + (pValue, degreesOfFreedom, fValue) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/ChiSquareTest.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/ChiSquareTest.scala index 159fc0a9885d2..a38a7c446ac52 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/stat/ChiSquareTest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/ChiSquareTest.scala @@ -21,12 +21,9 @@ import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vectors => OldVectors} -import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} -import org.apache.spark.mllib.stat.{Statistics => OldStatistics} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row} -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.types.DoubleType +import org.apache.spark.mllib.stat.test.{ChiSqTest => OldChiSqTest} +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions._ /** @@ -38,12 +35,6 @@ import org.apache.spark.sql.types.DoubleType @Since("2.2.0") object ChiSquareTest { - /** Used to construct output schema of tests */ - private case class ChiSquareResult( - pValues: Vector, - degreesOfFreedom: Array[Int], - statistics: Vector) - /** * Conduct Pearson's independence test for every feature against the label. For each feature, the * (feature, label) pairs are converted into a contingency matrix for which the Chi-squared @@ -64,18 +55,7 @@ object ChiSquareTest { */ @Since("2.2.0") def test(dataset: DataFrame, featuresCol: String, labelCol: String): DataFrame = { - val spark = dataset.sparkSession - import spark.implicits._ - - SchemaUtils.checkColumnType(dataset.schema, featuresCol, new VectorUDT) - SchemaUtils.checkNumericType(dataset.schema, labelCol) - val rdd = dataset.select(col(labelCol).cast("double"), col(featuresCol)).as[(Double, Vector)] - .rdd.map { case (label, features) => OldLabeledPoint(label, OldVectors.fromML(features)) } - val testResults = OldStatistics.chiSqTest(rdd) - val pValues: Vector = Vectors.dense(testResults.map(_.pValue)) - val degreesOfFreedom: Array[Int] = testResults.map(_.degreesOfFreedom) - val statistics: Vector = Vectors.dense(testResults.map(_.statistic)) - spark.createDataFrame(Seq(ChiSquareResult(pValues, degreesOfFreedom, statistics))) + test(dataset, featuresCol, labelCol, false) } /** @@ -83,25 +63,42 @@ object ChiSquareTest { * Real-valued features will be treated as categorical for each distinct value. * @param featuresCol Name of features column in dataset, of type `Vector` (`VectorUDT`) * @param labelCol Name of label column in dataset, of any numerical type - * @return Array containing the SelectionTestResult for every feature against the label. + * @param flatten If false, the returned DataFrame contains only a single Row, otherwise, one + * row per feature. */ @Since("3.1.0") - def testChiSquare( - dataset: Dataset[_], + def test( + dataset: DataFrame, featuresCol: String, - labelCol: String): Array[SelectionTestResult] = { + labelCol: String, + flatten: Boolean): DataFrame = { + SchemaUtils.checkColumnType(dataset.schema, featuresCol, new VectorUDT) + SchemaUtils.checkNumericType(dataset.schema, labelCol) val spark = dataset.sparkSession + import spark.implicits._ - SchemaUtils.checkColumnType(dataset.schema, featuresCol, new VectorUDT) - SchemaUtils.checkNumericType(dataset.schema, labelCol) - val input: RDD[OldLabeledPoint] = - dataset.select(col(labelCol).cast(DoubleType), col(featuresCol)).rdd - .map { - case Row(label: Double, features: Vector) => - OldLabeledPoint(label, OldVectors.fromML(features)) - } - val chiTestResult = OldStatistics.chiSqTest(input) - chiTestResult.map(r => new ChiSqTestResult(r.pValue, r.degreesOfFreedom, r.statistic)) + val data = dataset.select(col(labelCol).cast("double"), col(featuresCol)).rdd + .map { case Row(label: Double, vec: Vector) => (label, OldVectors.fromML(vec)) } + + val resultDF = OldChiSqTest.computeChiSquared(data) + .map { case (col, pValue, degreesOfFreedom, statistic, _) => + (col, pValue, degreesOfFreedom, statistic) + }.toDF("featureIndex", "pValue", "degreesOfFreedom", "statistic") + + if (flatten) { + resultDF + } else { + resultDF.groupBy() + .agg(collect_list(struct("*"))) + .as[Seq[(Int, Double, Int, Double)]] + .map { seq => + val results = seq.toArray.sortBy(_._1) + val pValues = Vectors.dense(results.map(_._2)) + val degreesOfFreedom = results.map(_._3) + val statistics = Vectors.dense(results.map(_._4)) + (pValues, degreesOfFreedom, statistics) + }.toDF("pValues", "degreesOfFreedom", "statistics") + } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/FValueTest.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/FValueTest.scala index edd8efad59feb..2c83c9811bab9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/stat/FValueTest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/FValueTest.scala @@ -44,7 +44,7 @@ object FValueTest { * @return DataFrame containing the test result for every feature against the label. * This DataFrame will contain a single Row with the following fields: * - `pValues: Vector` - * - `degreesOfFreedom: Array[Int]` + * - `degreesOfFreedom: Array[Long]` * - `fValues: Vector` * Each of these fields has one value per feature. */ @@ -52,22 +52,22 @@ object FValueTest { def test(dataset: DataFrame, featuresCol: String, labelCol: String): DataFrame = { val spark = dataset.sparkSession val testResults = testRegression(dataset, featuresCol, labelCol) - val pValues: Vector = Vectors.dense(testResults.map(_.pValue)) - val degreesOfFreedom: Array[Long] = testResults.map(_.degreesOfFreedom) - val fValues: Vector = Vectors.dense(testResults.map(_.statistic)) - spark.createDataFrame( - Seq(new FValueResult(pValues, degreesOfFreedom, fValues))) + val pValues = Vectors.dense(testResults.map(_.pValue)) + val degreesOfFreedom = testResults.map(_.degreesOfFreedom) + val fValues = Vectors.dense(testResults.map(_.statistic)) + spark.createDataFrame(Seq(FValueResult(pValues, degreesOfFreedom, fValues))) } /** * @param dataset DataFrame of continuous labels and continuous features. * @param featuresCol Name of features column in dataset, of type `Vector` (`VectorUDT`) * @param labelCol Name of label column in dataset, of any numerical type - * @return Array containing the FRegressionTestResult for every feature against the label. + * @return Array containing the FValueTestResult for every feature against the label. */ - @Since("3.1.0") - private[ml] def testRegression(dataset: Dataset[_], featuresCol: String, labelCol: String): - Array[SelectionTestResult] = { + private[ml] def testRegression( + dataset: Dataset[_], + featuresCol: String, + labelCol: String): Array[SelectionTestResult] = { val spark = dataset.sparkSession import spark.implicits._ @@ -82,7 +82,7 @@ object FValueTest { .select("summary.mean", "summary.std", "yMean", "yStd", "summary.count") .first() - val labeledPointRdd = dataset.select(col("label").cast("double"), col("features")) + val labeledPointRdd = dataset.select(col(labelCol).cast("double"), col(featuresCol)) .as[(Double, Vector)].rdd val numFeatures = xMeans.size @@ -113,17 +113,15 @@ object FValueTest { } array1 } - var fTestResultArray = new Array[SelectionTestResult](numFeatures) val fd = new FDistribution(1, degreesOfFreedom) - for (i <- 0 until numFeatures) { + Array.tabulate(numFeatures) { i => // Cov(X,Y) = Sum(((Xi - Avg(X)) * ((Yi-Avg(Y))) / (N-1) val covariance = sumForCov (i) / (numSamples - 1) val corr = covariance / (yStd * xStd(i)) val fValue = corr * corr / (1 - corr * corr) * degreesOfFreedom val pValue = 1.0 - fd.cumulativeProbability(fValue) - fTestResultArray(i) = new FValueTestResult(pValue, degreesOfFreedom, fValue) + new FValueTestResult(pValue, degreesOfFreedom, fValue) } - fTestResultArray } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/SelectionTestResult.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/SelectionTestResult.scala index 18a7a79a2c2ca..7a06e14e3222d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/stat/SelectionTestResult.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/SelectionTestResult.scala @@ -99,3 +99,19 @@ class FValueTestResult private[stat] ( s"F Value = $statistic \n" } } + +/** + * Object containing the test results for the ANOVA classification test. + */ +@Since("3.1.0") +class ANOVATestResult private[stat] ( + override val pValue: Double, + override val degreesOfFreedom: Long, + override val statistic: Double) extends SelectionTestResult { + + override def toString: String = { + "ANOVA Regression test summary:\n" + + super.toString + + s"F Value = $statistic \n" + } +} 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 a273cd71a0beb..7e2c287f146fb 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 @@ -36,7 +36,7 @@ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} /** * Parameters for Decision Tree-based algorithms. * - * Note: Marked as private and DeveloperApi since this may be made public in the future. + * Note: Marked as private since this may be made public in the future. */ private[ml] trait DecisionTreeParams extends PredictorParams with HasCheckpointInterval with HasSeed with HasWeightCol { @@ -320,7 +320,7 @@ private[spark] object TreeEnsembleParams { /** * Parameters for Decision Tree-based ensemble algorithms. * - * Note: Marked as private and DeveloperApi since this may be made public in the future. + * Note: Marked as private since this may be made public in the future. */ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { @@ -477,7 +477,7 @@ private[ml] trait RandomForestRegressorParams /** * Parameters for Gradient-Boosted Tree algorithms. * - * Note: Marked as private and DeveloperApi since this may be made public in the future. + * Note: Marked as private since this may be made public in the future. */ private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter with HasStepSize with HasValidationIndicatorCol { diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/Identifiable.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Identifiable.scala index bd213e7362e94..653ffb765f610 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/Identifiable.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Identifiable.scala @@ -19,18 +19,13 @@ package org.apache.spark.ml.util import java.util.UUID -import org.apache.spark.annotation.DeveloperApi - /** - * :: DeveloperApi :: - * * Trait for an object with an immutable unique ID that identifies itself and its derivatives. * * WARNING: There have not yet been final discussions on this API, so it may be broken in future * releases. */ -@DeveloperApi trait Identifiable { /** @@ -41,10 +36,7 @@ trait Identifiable { override def toString: String = uid } -/** - * :: DeveloperApi :: - */ -@DeveloperApi + object Identifiable { /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index dd0c40f33ff2a..fec05ccf15c7c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -31,7 +31,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.annotation.{DeveloperApi, Since, Unstable} +import org.apache.spark.annotation.{Since, Unstable} import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.classification.{OneVsRest, OneVsRestModel} @@ -301,8 +301,6 @@ trait GeneralMLWritable extends MLWritable { } /** - * :: DeveloperApi :: - * * Helper trait for making simple `Params` types writable. If a `Params` class stores * all data as [[org.apache.spark.ml.param.Param]] values, then extending this trait will provide * a default implementation of writing saved instances of the class. @@ -311,7 +309,6 @@ trait GeneralMLWritable extends MLWritable { * * @see `DefaultParamsReadable`, the counterpart to this trait */ -@DeveloperApi trait DefaultParamsWritable extends MLWritable { self: Params => override def write: MLWriter = new DefaultParamsWriter(this) @@ -360,8 +357,6 @@ trait MLReadable[T] { /** - * :: DeveloperApi :: - * * Helper trait for making simple `Params` types readable. If a `Params` class stores * all data as [[org.apache.spark.ml.param.Param]] values, then extending this trait will provide * a default implementation of reading saved instances of the class. @@ -371,7 +366,6 @@ trait MLReadable[T] { * @tparam T ML instance type * @see `DefaultParamsWritable`, the counterpart to this trait */ -@DeveloperApi trait DefaultParamsReadable[T] extends MLReadable[T] { override def read: MLReader[T] = new DefaultParamsReader[T] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index e858ac9844a24..8c43c0b468526 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -21,7 +21,7 @@ import java.util.Locale import breeze.linalg.{DenseVector => BDV} -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.graphx._ import org.apache.spark.internal.Logging @@ -281,21 +281,15 @@ class LDA private ( /** - * :: DeveloperApi :: - * * LDAOptimizer used to perform the actual calculation */ @Since("1.4.0") - @DeveloperApi def getOptimizer: LDAOptimizer = ldaOptimizer /** - * :: DeveloperApi :: - * * LDAOptimizer used to perform the actual calculation (default = EMLDAOptimizer) */ @Since("1.4.0") - @DeveloperApi def setOptimizer(optimizer: LDAOptimizer): this.type = { this.ldaOptimizer = optimizer this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala index dc90f6c3e3885..b2742ee6ecb5b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -23,7 +23,7 @@ import breeze.linalg.{all, normalize, sum, DenseMatrix => BDM, DenseVector => BD import breeze.numerics.{abs, exp, trigamma} import breeze.stats.distributions.{Gamma, RandBasis} -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.graphx._ import org.apache.spark.graphx.util.PeriodicGraphCheckpointer import org.apache.spark.internal.Logging @@ -33,13 +33,10 @@ import org.apache.spark.storage.StorageLevel /** - * :: DeveloperApi :: - * * An LDAOptimizer specifies which optimization/learning/inference algorithm to use, and it can * hold optimizer-specific parameters for users to set. */ @Since("1.4.0") -@DeveloperApi trait LDAOptimizer { /* @@ -62,8 +59,6 @@ trait LDAOptimizer { } /** - * :: DeveloperApi :: - * * Optimizer for EM algorithm which stores data + parameter graph, plus algorithm parameters. * * Currently, the underlying implementation uses Expectation-Maximization (EM), implemented @@ -78,7 +73,6 @@ trait LDAOptimizer { * "On Smoothing and Inference for Topic Models." UAI, 2009. */ @Since("1.4.0") -@DeveloperApi final class EMLDAOptimizer extends LDAOptimizer { import LDA._ @@ -253,8 +247,6 @@ final class EMLDAOptimizer extends LDAOptimizer { /** - * :: DeveloperApi :: - * * An online optimizer for LDA. The Optimizer implements the Online variational Bayes LDA * algorithm, which processes a subset of the corpus on each iteration, and updates the term-topic * distribution adaptively. @@ -263,7 +255,6 @@ final class EMLDAOptimizer extends LDAOptimizer { * Hoffman, Blei and Bach, "Online Learning for Latent Dirichlet Allocation." NIPS, 2010. */ @Since("1.4.0") -@DeveloperApi final class OnlineLDAOptimizer extends LDAOptimizer with Logging { // LDA common parameters diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index 21e01ef36814f..78c974e22f2cf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.feature -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.{StandardScalerModel => NewStandardScalerModel} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} @@ -96,22 +96,14 @@ class StandardScalerModel @Since("1.3.0") ( @Since("1.3.0") def this(std: Vector) = this(std, null) - /** - * :: DeveloperApi :: - */ @Since("1.3.0") - @DeveloperApi def setWithMean(withMean: Boolean): this.type = { require(!(withMean && this.mean == null), "cannot set withMean to true while mean is null") this.withMean = withMean this } - /** - * :: DeveloperApi :: - */ @Since("1.3.0") - @DeveloperApi def setWithStd(withStd: Boolean): this.type = { require(!(withStd && this.std == null), "cannot set withStd to true while std is null") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala index 9db725097ae90..ac58ffe3e5509 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala @@ -17,17 +17,15 @@ package org.apache.spark.mllib.feature -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * Trait for transformation of a vector */ @Since("1.1.0") -@DeveloperApi trait VectorTransformer extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 08086ceff9ef4..da486010cfa9e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -29,8 +29,9 @@ private[spark] object BLAS extends Serializable with Logging { @transient private var _f2jBLAS: NetlibBLAS = _ @transient private var _nativeBLAS: NetlibBLAS = _ + private val nativeL1Threshold: Int = 256 - // For level-1 routines, we use Java implementation. + // For level-1 function dspmv, use f2jBLAS for better performance. private[mllib] def f2jBLAS: NetlibBLAS = { if (_f2jBLAS == null) { _f2jBLAS = new F2jBLAS @@ -38,6 +39,14 @@ private[spark] object BLAS extends Serializable with Logging { _f2jBLAS } + private[mllib] def getBLAS(vectorSize: Int): NetlibBLAS = { + if (vectorSize < nativeL1Threshold) { + f2jBLAS + } else { + nativeBLAS + } + } + /** * y += a * x */ @@ -65,7 +74,7 @@ private[spark] object BLAS extends Serializable with Logging { */ private def axpy(a: Double, x: DenseVector, y: DenseVector): Unit = { val n = x.size - f2jBLAS.daxpy(n, a, x.values, 1, y.values, 1) + getBLAS(n).daxpy(n, a, x.values, 1, y.values, 1) } /** @@ -96,7 +105,7 @@ private[spark] object BLAS extends Serializable with Logging { private[spark] def axpy(a: Double, X: DenseMatrix, Y: DenseMatrix): Unit = { require(X.numRows == Y.numRows && X.numCols == Y.numCols, "Dimension mismatch: " + s"size(X) = ${(X.numRows, X.numCols)} but size(Y) = ${(Y.numRows, Y.numCols)}.") - f2jBLAS.daxpy(X.numRows * X.numCols, a, X.values, 1, Y.values, 1) + getBLAS(X.values.length).daxpy(X.numRows * X.numCols, a, X.values, 1, Y.values, 1) } /** @@ -125,7 +134,7 @@ private[spark] object BLAS extends Serializable with Logging { */ private def dot(x: DenseVector, y: DenseVector): Double = { val n = x.size - f2jBLAS.ddot(n, x.values, 1, y.values, 1) + getBLAS(n).ddot(n, x.values, 1, y.values, 1) } /** @@ -220,16 +229,16 @@ private[spark] object BLAS extends Serializable with Logging { def scal(a: Double, x: Vector): Unit = { x match { case sx: SparseVector => - f2jBLAS.dscal(sx.values.length, a, sx.values, 1) + getBLAS(sx.values.length).dscal(sx.values.length, a, sx.values, 1) case dx: DenseVector => - f2jBLAS.dscal(dx.values.length, a, dx.values, 1) + getBLAS(dx.size).dscal(dx.values.length, a, dx.values, 1) case _ => throw new IllegalArgumentException(s"scal doesn't support vector type ${x.getClass}.") } } // For level-3 routines, we use the native BLAS. - private def nativeBLAS: NetlibBLAS = { + private[mllib] def nativeBLAS: NetlibBLAS = { if (_nativeBLAS == null) { _nativeBLAS = NativeBLAS } @@ -356,7 +365,7 @@ private[spark] object BLAS extends Serializable with Logging { if (alpha == 0.0 && beta == 1.0) { logDebug("gemm: alpha is equal to 0 and beta is equal to 1. Returning C.") } else if (alpha == 0.0) { - f2jBLAS.dscal(C.values.length, beta, C.values, 1) + getBLAS(C.values.length).dscal(C.values.length, beta, C.values, 1) } else { A match { case sparse: SparseMatrix => gemm(alpha, sparse, B, beta, C) @@ -462,7 +471,7 @@ private[spark] object BLAS extends Serializable with Logging { } else { // Scale matrix first if `beta` is not equal to 1.0 if (beta != 1.0) { - f2jBLAS.dscal(C.values.length, beta, C.values, 1) + getBLAS(C.values.length).dscal(C.values.length, beta, C.values, 1) } // Perform matrix multiplication and add to C. The rows of A are multiplied by the columns of // B, and added to C. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 46a18295d839b..cdeed33c92381 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -17,16 +17,13 @@ package org.apache.spark.mllib.optimization -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.{axpy, dot, scal} import org.apache.spark.mllib.util.MLUtils /** - * :: DeveloperApi :: * Class used to compute the gradient for a loss function, given a single data point. */ -@DeveloperApi abstract class Gradient extends Serializable { /** * Compute the gradient and loss given the features of a single data point. @@ -58,7 +55,6 @@ abstract class Gradient extends Serializable { } /** - * :: DeveloperApi :: * Compute gradient and loss for a multinomial logistic loss function, as used * in multi-class classification (it is also used in binary logistic regression). * @@ -162,7 +158,6 @@ abstract class Gradient extends Serializable { * Multinomial Logistic Regression. By default, it is binary logistic regression * so numClasses will be set to 2. */ -@DeveloperApi class LogisticGradient(numClasses: Int) extends Gradient { def this() = this(2) @@ -275,13 +270,11 @@ class LogisticGradient(numClasses: Int) extends Gradient { } /** - * :: DeveloperApi :: * Compute gradient and loss for a Least-squared loss function, as used in linear regression. * This is correct for the averaged least squares loss function (mean squared error) * L = 1/2n ||A weights-y||^2 * See also the documentation for the precise formulation. */ -@DeveloperApi class LeastSquaresGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val diff = dot(data, weights) - label @@ -303,13 +296,11 @@ class LeastSquaresGradient extends Gradient { } /** - * :: DeveloperApi :: * Compute gradient and loss for a Hinge loss function, as used in SVM binary classification. * See also the documentation for the precise formulation. * * @note This assumes that the labels are {0,1} */ -@DeveloperApi class HingeGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val dotProduct = dot(data, weights) 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 b65b95d894413..1336ffd2f7d5e 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 @@ -21,7 +21,6 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{norm, DenseVector => BDV} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD @@ -124,13 +123,11 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va } /** - * :: DeveloperApi :: * Runs gradient descent on the given training data. * @param data training data * @param initialWeights initial weights * @return solution vector */ - @DeveloperApi def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { val (weights, _) = GradientDescent.runMiniBatchSGD( data, @@ -148,10 +145,8 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va } /** - * :: DeveloperApi :: * Top-level method to run gradient descent. */ -@DeveloperApi object GradientDescent extends Logging { /** * Run stochastic gradient descent (SGD) in parallel using mini batches. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 65c1159723c80..1ee9241104f87 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -22,21 +22,18 @@ import scala.collection.mutable import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.axpy import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * Class used to solve an optimization problem using Limited-memory BFGS. * Reference: * Wikipedia on Limited-memory BFGS * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. */ -@DeveloperApi class LBFGS(private var gradient: Gradient, private var updater: Updater) extends Optimizer with Logging { @@ -154,10 +151,8 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) } /** - * :: DeveloperApi :: * Top-level method to run L-BFGS. */ -@DeveloperApi object LBFGS extends Logging { /** * Run Limited-memory BFGS (L-BFGS) in parallel. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index d8e56720967d8..582b520e08eee 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -17,15 +17,12 @@ package org.apache.spark.mllib.optimization -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * Trait for optimization problem solvers. */ -@DeveloperApi trait Optimizer extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index 142f0ec6b9021..1865dd96ce72d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -21,11 +21,9 @@ import scala.math._ import breeze.linalg.{axpy => brzAxpy, norm => brzNorm, Vector => BV} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{Vector, Vectors} /** - * :: DeveloperApi :: * Class used to perform steps (weight update) using Gradient Descent methods. * * For general minimization problems, or for regularized problems of the form @@ -37,7 +35,6 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} * The updater is responsible to also perform the update coming from the * regularization term R(w) (if any regularization is used). */ -@DeveloperApi abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize, iteration number and @@ -62,11 +59,9 @@ abstract class Updater extends Serializable { } /** - * :: DeveloperApi :: * A simple updater for gradient descent *without* any regularization. * Uses a step-size decreasing with the square root of the number of iterations. */ -@DeveloperApi class SimpleUpdater extends Updater { override def compute( weightsOld: Vector, @@ -83,7 +78,6 @@ class SimpleUpdater extends Updater { } /** - * :: DeveloperApi :: * Updater for L1 regularized problems. * R(w) = ||w||_1 * Uses a step-size decreasing with the square root of the number of iterations. @@ -101,7 +95,6 @@ class SimpleUpdater extends Updater { * * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) */ -@DeveloperApi class L1Updater extends Updater { override def compute( weightsOld: Vector, @@ -128,12 +121,10 @@ class L1Updater extends Updater { } /** - * :: DeveloperApi :: * Updater for L2 regularized problems. * R(w) = 1/2 ||w||^2 * Uses a step-size decreasing with the square root of the number of iterations. */ -@DeveloperApi class SquaredL2Updater extends Updater { override def compute( weightsOld: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala index 340386c7e7bec..32471b0a64d9e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -23,16 +23,14 @@ import javax.xml.transform.stream.StreamResult import org.jpmml.model.JAXBUtil import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory /** - * :: DeveloperApi :: * Export model to the PMML format * Predictive Model Markup Language (PMML) is an XML-based file format * developed by the Data Mining Group (www.dmg.org). */ -@DeveloperApi @Since("1.4.0") trait PMMLExportable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index d3b548832bb21..3a2ecf17d88d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -19,14 +19,12 @@ package org.apache.spark.mllib.random import org.apache.commons.math3.distribution._ -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.util.random.{Pseudorandom, XORShiftRandom} /** - * :: DeveloperApi :: * Trait for random data generators that generate i.i.d. data. */ -@DeveloperApi @Since("1.1.0") trait RandomDataGenerator[T] extends Pseudorandom with Serializable { @@ -45,10 +43,8 @@ trait RandomDataGenerator[T] extends Pseudorandom with Serializable { } /** - * :: DeveloperApi :: * Generates i.i.d. samples from U[0.0, 1.0] */ -@DeveloperApi @Since("1.1.0") class UniformGenerator extends RandomDataGenerator[Double] { @@ -68,10 +64,8 @@ class UniformGenerator extends RandomDataGenerator[Double] { } /** - * :: DeveloperApi :: * Generates i.i.d. samples from the standard normal distribution. */ -@DeveloperApi @Since("1.1.0") class StandardNormalGenerator extends RandomDataGenerator[Double] { @@ -91,12 +85,10 @@ class StandardNormalGenerator extends RandomDataGenerator[Double] { } /** - * :: DeveloperApi :: * Generates i.i.d. samples from the Poisson distribution with the given mean. * * @param mean mean for the Poisson distribution. */ -@DeveloperApi @Since("1.1.0") class PoissonGenerator @Since("1.1.0") ( @Since("1.1.0") val mean: Double) extends RandomDataGenerator[Double] { @@ -116,12 +108,10 @@ class PoissonGenerator @Since("1.1.0") ( } /** - * :: DeveloperApi :: * Generates i.i.d. samples from the exponential distribution with the given mean. * * @param mean mean for the exponential distribution. */ -@DeveloperApi @Since("1.3.0") class ExponentialGenerator @Since("1.3.0") ( @Since("1.3.0") val mean: Double) extends RandomDataGenerator[Double] { @@ -141,13 +131,11 @@ class ExponentialGenerator @Since("1.3.0") ( } /** - * :: DeveloperApi :: * Generates i.i.d. samples from the gamma distribution with the given shape and scale. * * @param shape shape for the gamma distribution. * @param scale scale for the gamma distribution */ -@DeveloperApi @Since("1.3.0") class GammaGenerator @Since("1.3.0") ( @Since("1.3.0") val shape: Double, @@ -168,14 +156,12 @@ class GammaGenerator @Since("1.3.0") ( } /** - * :: DeveloperApi :: * Generates i.i.d. samples from the log normal distribution with the * given mean and standard deviation. * * @param mean mean for the log normal distribution. * @param std standard deviation for the log normal distribution */ -@DeveloperApi @Since("1.3.0") class LogNormalGenerator @Since("1.3.0") ( @Since("1.3.0") val mean: Double, @@ -196,14 +182,12 @@ class LogNormalGenerator @Since("1.3.0") ( } /** - * :: DeveloperApi :: * Generates i.i.d. samples from the Weibull distribution with the * given shape and scale parameter. * * @param alpha shape parameter for the Weibull distribution. * @param beta scale parameter for the Weibull distribution. */ -@DeveloperApi class WeibullGenerator( val alpha: Double, val beta: Double) extends RandomDataGenerator[Double] { 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 258b1763bba86..eacccfcc4d59f 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 @@ -20,7 +20,7 @@ package org.apache.spark.mllib.random import scala.reflect.ClassTag import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD, JavaSparkContext} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.mllib.linalg.Vector @@ -372,7 +372,6 @@ object RandomRDDs { /** - * :: DeveloperApi :: * Generates an RDD comprised of `i.i.d.` samples produced by the input RandomDataGenerator. * * @param sc SparkContext used to create the RDD. @@ -382,7 +381,6 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[T] comprised of `i.i.d.` samples produced by generator. */ - @DeveloperApi @Since("1.1.0") def randomRDD[T: ClassTag]( sc: SparkContext, @@ -394,7 +392,6 @@ object RandomRDDs { } /** - * :: DeveloperApi :: * Generates an RDD comprised of `i.i.d.` samples produced by the input RandomDataGenerator. * * @param jsc JavaSparkContext used to create the RDD. @@ -404,7 +401,6 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[T] comprised of `i.i.d.` samples produced by generator. */ - @DeveloperApi @Since("1.6.0") def randomJavaRDD[T]( jsc: JavaSparkContext, @@ -418,10 +414,8 @@ object RandomRDDs { } /** - * :: DeveloperApi :: * `RandomRDDs.randomJavaRDD` with the default seed. */ - @DeveloperApi @Since("1.6.0") def randomJavaRDD[T]( jsc: JavaSparkContext, @@ -432,10 +426,8 @@ object RandomRDDs { } /** - * :: DeveloperApi :: * `RandomRDDs.randomJavaRDD` with the default seed & numPartitions */ - @DeveloperApi @Since("1.6.0") def randomJavaRDD[T]( jsc: JavaSparkContext, @@ -831,7 +823,6 @@ object RandomRDDs { /** - * :: DeveloperApi :: * Generates an RDD[Vector] with vectors containing `i.i.d.` samples produced by the * input RandomDataGenerator. * @@ -843,7 +834,6 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing `i.i.d.` samples produced by generator. */ - @DeveloperApi @Since("1.1.0") def randomVectorRDD(sc: SparkContext, generator: RandomDataGenerator[Double], @@ -856,10 +846,8 @@ object RandomRDDs { } /** - * :: DeveloperApi :: * Java-friendly version of `RandomRDDs.randomVectorRDD`. */ - @DeveloperApi @Since("1.6.0") def randomJavaVectorRDD( jsc: JavaSparkContext, @@ -871,11 +859,9 @@ object RandomRDDs { randomVectorRDD(jsc.sc, generator, numRows, numCols, numPartitions, seed).toJavaRDD() } - /** - * :: DeveloperApi :: + /** :: * `RandomRDDs.randomJavaVectorRDD` with the default seed. */ - @DeveloperApi @Since("1.6.0") def randomJavaVectorRDD( jsc: JavaSparkContext, @@ -887,10 +873,8 @@ object RandomRDDs { } /** - * :: DeveloperApi :: * `RandomRDDs.randomJavaVectorRDD` with the default number of partitions and the default seed. */ - @DeveloperApi @Since("1.6.0") def randomJavaVectorRDD( jsc: JavaSparkContext, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala index e28e1af5b0a26..8f78bcc15347f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala @@ -20,15 +20,12 @@ package org.apache.spark.mllib.rdd import scala.language.implicitConversions import scala.reflect.ClassTag -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.util.BoundedPriorityQueue /** - * :: DeveloperApi :: * Machine learning specific Pair RDD functions. */ -@DeveloperApi class MLPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Serializable { /** * Returns the top k (largest) elements for each key from this RDD as defined by the specified @@ -51,10 +48,6 @@ class MLPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Se } } -/** - * :: DeveloperApi :: - */ -@DeveloperApi object MLPairRDDFunctions { /** Implicit conversion from a pair RDD to MLPairRDDFunctions. */ implicit def fromPairRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): MLPairRDDFunctions[K, V] = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala index 32e6ecf6308e0..1a13cbb5191c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -20,14 +20,11 @@ package org.apache.spark.mllib.rdd import scala.language.implicitConversions import scala.reflect.ClassTag -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * Machine learning specific RDD functions. */ -@DeveloperApi class RDDFunctions[T: ClassTag](self: RDD[T]) extends Serializable { /** @@ -54,10 +51,7 @@ class RDDFunctions[T: ClassTag](self: RDD[T]) extends Serializable { } -/** - * :: DeveloperApi :: - */ -@DeveloperApi + object RDDFunctions { /** Implicit conversion from an RDD to RDDFunctions. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index f3f15ba0d0f2c..1ae0b580a3716 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.recommendation -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.ml.recommendation.{ALS => NewALS} @@ -195,12 +195,10 @@ class ALS private ( } /** - * :: DeveloperApi :: * Sets storage level for intermediate RDDs (user/product in/out links). The default value is * `MEMORY_AND_DISK`. Users can change it to a serialized storage, e.g., `MEMORY_AND_DISK_SER` and * set `spark.rdd.compress` to `true` to reduce the space requirement, at the cost of speed. */ - @DeveloperApi @Since("1.1.0") def setIntermediateRDDStorageLevel(storageLevel: StorageLevel): this.type = { require(storageLevel != StorageLevel.NONE, @@ -210,13 +208,11 @@ class ALS private ( } /** - * :: DeveloperApi :: * Sets storage level for final RDDs (user/product used in MatrixFactorizationModel). The default * value is `MEMORY_AND_DISK`. Users can change it to a serialized storage, e.g. * `MEMORY_AND_DISK_SER` and set `spark.rdd.compress` to `true` to reduce the space requirement, * at the cost of speed. */ - @DeveloperApi @Since("1.3.0") def setFinalRDDStorageLevel(storageLevel: StorageLevel): this.type = { this.finalRDDStorageLevel = storageLevel @@ -224,14 +220,12 @@ class ALS private ( } /** - * :: DeveloperApi :: * Set period (in iterations) between checkpoints (default = 10). Checkpointing helps with * recovery (when nodes fail) and StackOverflow exceptions caused by long lineage. It also helps * with eliminating temporary shuffle files on disk, which can be important when there are many * ALS iterations. If the checkpoint directory is not set in [[org.apache.spark.SparkContext]], * this setting is ignored. */ - @DeveloperApi @Since("1.4.0") def setCheckpointInterval(checkpointInterval: Int): this.type = { this.checkpointInterval = checkpointInterval diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index ef3ff2b98a5e6..6d0b3fa791d6e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkException -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.mllib.feature.StandardScaler import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -28,7 +28,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel /** - * :: DeveloperApi :: * GeneralizedLinearModel (GLM) represents a model trained using * GeneralizedLinearAlgorithm. GLMs consist of a weight vector and * an intercept. @@ -38,7 +37,6 @@ import org.apache.spark.storage.StorageLevel * */ @Since("0.8.0") -@DeveloperApi abstract class GeneralizedLinearModel @Since("1.0.0") ( @Since("1.0.0") val weights: Vector, @Since("0.8.0") val intercept: Double) @@ -94,13 +92,11 @@ abstract class GeneralizedLinearModel @Since("1.0.0") ( } /** - * :: DeveloperApi :: * GeneralizedLinearAlgorithm implements methods to train a Generalized Linear Model (GLM). * This class should be extended with an Optimizer to create a new GLM. * */ @Since("0.8.0") -@DeveloperApi abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] extends Logging with Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index f44c8fe351459..eb83f9a6b75af 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.regression import scala.reflect.ClassTag -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector @@ -27,7 +27,6 @@ import org.apache.spark.streaming.api.java.{JavaDStream, JavaPairDStream} import org.apache.spark.streaming.dstream.DStream /** - * :: DeveloperApi :: * StreamingLinearAlgorithm implements methods for continuously * training a generalized linear model on streaming data, * and using it for prediction on (possibly different) streaming data. @@ -56,7 +55,6 @@ import org.apache.spark.streaming.dstream.DStream * */ @Since("1.1.0") -@DeveloperApi abstract class StreamingLinearAlgorithm[ M <: GeneralizedLinearModel, A <: GeneralizedLinearAlgorithm[M]] extends Logging { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 00c1da66d7df6..829cce6eac73e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.stat -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{Vector, Vectors} /** - * :: DeveloperApi :: * MultivariateOnlineSummarizer implements [[MultivariateStatisticalSummary]] to compute the mean, * variance, minimum, maximum, counts, and nonzero counts for instances in sparse or dense vector * format in an online fashion. @@ -41,7 +40,6 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} * Reliability weights (Wikipedia). */ @Since("1.1.0") -@DeveloperApi class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with Serializable { private var n = 0 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index f34c22915ae15..310eb8639b3c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -19,12 +19,11 @@ package org.apache.spark.mllib.stat.distribution import breeze.linalg.{diag, eigSym, max, DenseMatrix => DBM, DenseVector => DBV, Vector => BV} -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.mllib.util.MLUtils /** - * :: DeveloperApi :: * This class provides basic functionality for a Multivariate Gaussian (Normal) Distribution. In * the event that the covariance matrix is singular, the density will be computed in a * reduced dimensional subspace under which the distribution is supported. @@ -35,7 +34,6 @@ import org.apache.spark.mllib.util.MLUtils * @param sigma The covariance matrix of the distribution */ @Since("1.3.0") -@DeveloperApi class MultivariateGaussian @Since("1.3.0") ( @Since("1.3.0") val mu: Vector, @Since("1.3.0") val sigma: Matrix) extends Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index a4e398d34347e..9f0832804f27f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -79,9 +79,20 @@ private[spark] object ChiSqTest extends Logging { * the independence test. * Returns an array containing the ChiSquaredTestResult for every feature against the label. */ - def chiSquaredFeatures(data: RDD[LabeledPoint], + def chiSquaredFeatures( + data: RDD[LabeledPoint], methodName: String = PEARSON.name): Array[ChiSqTestResult] = { - data.first().features match { + computeChiSquared(data.map(l => (l.label, l.features)), methodName) + .collect().sortBy(_._1) + .map { case (_, pValue, degreesOfFreedom, statistic, nullHypothesis) => + new ChiSqTestResult(pValue, degreesOfFreedom, statistic, methodName, nullHypothesis) + } + } + + private[spark] def computeChiSquared( + data: RDD[(Double, Vector)], + methodName: String = PEARSON.name): RDD[(Int, Double, Int, Double, String)] = { + data.first()._2 match { case dv: DenseVector => chiSquaredDenseFeatures(data, dv.size, methodName) case sv: SparseVector => @@ -89,154 +100,110 @@ private[spark] object ChiSqTest extends Logging { } } - private def chiSquaredDenseFeatures(data: RDD[LabeledPoint], + private def chiSquaredDenseFeatures( + data: RDD[(Double, Vector)], numFeatures: Int, - methodName: String = PEARSON.name): Array[ChiSqTestResult] = { - data.flatMap { case LabeledPoint(label, features) => - require(features.size == numFeatures) - features.iterator.map { case (col, value) => - (col, (value, label)) - } + methodName: String = PEARSON.name): RDD[(Int, Double, Int, Double, String)] = { + data.flatMap { case (label, features) => + require(features.size == numFeatures, + s"Number of features must be $numFeatures but got ${features.size}") + features.iterator.map { case (col, value) => (col, (label, value)) } }.aggregateByKey(new OpenHashMap[(Double, Double), Long])( - seqOp = { case (count, t) => - count.changeValue(t, 1L, _ + 1L) - count + seqOp = { case (counts, t) => + counts.changeValue(t, 1L, _ + 1L) + counts }, - combOp = { case (count1, count2) => - count2.iterator.foreach { case (t, c) => - count1.changeValue(t, c, _ + c) - } - count1 - } - ).map { case (col, count) => - val label2Index = count.iterator.map(_._1._2).toArray.distinct.sorted.zipWithIndex.toMap - val numLabels = label2Index.size - if (numLabels > maxCategories) { - throw new SparkException(s"Chi-square test expect factors (categorical values) but " - + s"found more than $maxCategories distinct label values.") - } - - val value2Index = count.iterator.map(_._1._1).toArray.distinct.sorted.zipWithIndex.toMap - val numValues = value2Index.size - if (numValues > maxCategories) { - throw new SparkException(s"Chi-square test expect factors (categorical values) but " - + s"found more than $maxCategories distinct values in column $col.") - } - - val contingency = new DenseMatrix(numValues, numLabels, - Array.ofDim[Double](numValues * numLabels)) - count.foreach { case ((value, label), c) => - val i = value2Index(value) - val j = label2Index(label) - contingency.update(i, j, c) + combOp = { case (counts1, counts2) => + counts2.foreach { case (t, c) => counts1.changeValue(t, c, _ + c) } + counts1 } - - val result = ChiSqTest.chiSquaredMatrix(contingency, methodName) + ).map { case (col, counts) => + val result = computeChiSq(counts.toMap, methodName, col) (col, result.pValue, result.degreesOfFreedom, result.statistic, result.nullHypothesis) - }.collect().sortBy(_._1).map { - case (_, pValue, degreesOfFreedom, statistic, nullHypothesis) => - new ChiSqTestResult(pValue, degreesOfFreedom, statistic, methodName, nullHypothesis) } } - private def chiSquaredSparseFeatures(data: RDD[LabeledPoint], + private def chiSquaredSparseFeatures( + data: RDD[(Double, Vector)], numFeatures: Int, - methodName: String = PEARSON.name): Array[ChiSqTestResult] = { - val labelCounts = data.map(_.label).countByValue() + methodName: String = PEARSON.name): RDD[(Int, Double, Int, Double, String)] = { + val labelCounts = data.map(_._1).countByValue().toMap + val numInstances = labelCounts.valuesIterator.sum val numLabels = labelCounts.size if (numLabels > maxCategories) { throw new SparkException(s"Chi-square test expect factors (categorical values) but " + s"found more than $maxCategories distinct label values.") } - val numInstances = labelCounts.valuesIterator.sum - val label2Index = labelCounts.keys.toArray.sorted.zipWithIndex.toMap - - val sc = data.sparkContext - val bcLabels = sc.broadcast((labelCounts, label2Index)) - - val results = data.flatMap { case LabeledPoint(label, features) => - require(features.size == numFeatures) - features.nonZeroIterator.map { case (col, value) => - (col, (value, label)) + val numParts = data.getNumPartitions + data.mapPartitionsWithIndex { case (pid, iter) => + iter.flatMap { case (label, features) => + require(features.size == numFeatures, + s"Number of features must be $numFeatures but got ${features.size}") + features.nonZeroIterator.map { case (col, value) => (col, (label, value)) } + } ++ { + // append this to make sure that all columns are taken into account + Iterator.range(pid, numFeatures, numParts).map(col => (col, null)) } }.aggregateByKey(new OpenHashMap[(Double, Double), Long])( - seqOp = { case (count, t) => - count.changeValue(t, 1L, _ + 1L) - count + seqOp = { case (counts, labelAndValue) => + if (labelAndValue != null) counts.changeValue(labelAndValue, 1L, _ + 1L) + counts }, - combOp = { case (count1, count2) => - count2.iterator.foreach { case (t, c) => - count1.changeValue(t, c, _ + c) - } - count1 + combOp = { case (counts1, counts2) => + counts2.foreach { case (t, c) => counts1.changeValue(t, c, _ + c) } + counts1 } - ).map { case (col, count) => - val (labelCounts, label2Index) = bcLabels.value - val nnz = count.iterator.map(_._2).sum + ).map { case (col, counts) => + val nnz = counts.iterator.map(_._2).sum require(numInstances >= nnz) - - val value2Index = if (numInstances == nnz) { - count.iterator.map(_._1._1).toArray.distinct.sorted.zipWithIndex.toMap - } else { - (count.iterator.map(_._1._1).toArray :+ 0.0).distinct.sorted.zipWithIndex.toMap - } - val numValues = value2Index.size - if (numValues > maxCategories) { - throw new SparkException(s"Chi-square test expect factors (categorical values) but " - + s"found more than $maxCategories distinct values in column $col.") - } - - val contingency = new DenseMatrix(numValues, numLabels, - Array.ofDim[Double](numValues * numLabels)) - count.foreach { case ((value, label), c) => - val i = value2Index(value) - val j = label2Index(label) - contingency.update(i, j, c) - } - if (numInstances != nnz) { - val nnz = count.iterator - .map { case ((_, label), c) => (label, c) } + if (numInstances > nnz) { + val labelNNZ = counts.iterator + .map { case ((label, _), c) => (label, c) } .toArray .groupBy(_._1) .mapValues(_.map(_._2).sum) - val i = value2Index(0.0) - label2Index.foreach { case (label, j) => - val countByLabel = labelCounts(label) - val nnzByLabel = nnz.getOrElse(label, 0L) + labelCounts.foreach { case (label, countByLabel) => + val nnzByLabel = labelNNZ.getOrElse(label, 0L) val nzByLabel = countByLabel - nnzByLabel - require(nzByLabel >= 0) - if (nzByLabel != 0) contingency.update(i, j, nzByLabel) + if (nzByLabel > 0) { + counts.update((label, 0.0), nzByLabel) + } } } - val result = ChiSqTest.chiSquaredMatrix(contingency, methodName) - (col, (result.pValue, result.degreesOfFreedom, result.statistic, result.nullHypothesis)) - }.collectAsMap() - - bcLabels.destroy() + val result = computeChiSq(counts.toMap, methodName, col) + (col, result.pValue, result.degreesOfFreedom, result.statistic, result.nullHypothesis) + } + } - val finalResults = Array.ofDim[ChiSqTestResult](numFeatures) - results.foreach { case (col, (pValue, degreesOfFreedom, statistic, nullHypothesis)) => - finalResults(col) = new ChiSqTestResult(pValue, degreesOfFreedom, statistic, - methodName, nullHypothesis) + private def computeChiSq( + counts: Map[(Double, Double), Long], + methodName: String, + col: Int): ChiSqTestResult = { + val label2Index = counts.iterator.map(_._1._1).toArray.distinct.sorted.zipWithIndex.toMap + val numLabels = label2Index.size + if (numLabels > maxCategories) { + throw new SparkException(s"Chi-square test expect factors (categorical values) but " + + s"found more than $maxCategories distinct label values.") } - if (results.size < numFeatures) { - // if some column only contains 0 values - val zeroContingency = new DenseMatrix(1, numLabels, Array.ofDim[Double](numLabels)) - labelCounts.foreach { case (label, c) => - val j = label2Index(label) - zeroContingency.update(0, j, c) - } - val zeroRes = ChiSqTest.chiSquaredMatrix(zeroContingency, methodName) + val value2Index = counts.iterator.map(_._1._2).toArray.distinct.sorted.zipWithIndex.toMap + val numValues = value2Index.size + if (numValues > maxCategories) { + throw new SparkException(s"Chi-square test expect factors (categorical values) but " + + s"found more than $maxCategories distinct values in column $col.") + } - Iterator.range(0, numFeatures) - .filterNot(results.contains) - .foreach (col => finalResults(col) = zeroRes) + val contingency = new DenseMatrix(numValues, numLabels, + Array.ofDim[Double](numValues * numLabels)) + counts.foreach { case ((label, value), c) => + val i = value2Index(value) + val j = label2Index(label) + contingency.update(i, j, c) } - finalResults + ChiSqTest.chiSquaredMatrix(contingency, methodName) } /* diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index d52cb03b74d1e..6e2732f7ae7aa 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since /** * Class for calculating entropy during multiclass classification. @@ -30,14 +30,12 @@ object Entropy extends Impurity { private[tree] def log2(x: Double) = scala.math.log(x) / _log2 /** - * :: DeveloperApi :: * information calculation for multiclass classification * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels * @return information value, or 0 if totalCount = 0 */ @Since("1.1.0") - @DeveloperApi override def calculate(counts: Array[Double], totalCount: Double): Double = { if (totalCount == 0) { return 0 @@ -57,7 +55,6 @@ object Entropy extends Impurity { } /** - * :: DeveloperApi :: * variance calculation * @param count number of instances * @param sum sum of labels @@ -65,7 +62,6 @@ object Entropy extends Impurity { * @return information value, or 0 if count = 0 */ @Since("1.0.0") - @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Entropy.calculate") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 913ffbbb2457a..5983118c05754 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since /** * Class for calculating the Gini impurity @@ -28,14 +28,12 @@ import org.apache.spark.annotation.{DeveloperApi, Since} object Gini extends Impurity { /** - * :: DeveloperApi :: * information calculation for multiclass classification * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels * @return information value, or 0 if totalCount = 0 */ @Since("1.1.0") - @DeveloperApi override def calculate(counts: Array[Double], totalCount: Double): Double = { if (totalCount == 0) { return 0 @@ -52,7 +50,6 @@ object Gini extends Impurity { } /** - * :: DeveloperApi :: * variance calculation * @param count number of instances * @param sum sum of labels @@ -60,7 +57,6 @@ object Gini extends Impurity { * @return information value, or 0 if count = 0 */ @Since("1.0.0") - @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Gini.calculate") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 491473490eba7..9b1d8d717d40f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.tree.impurity import java.util.Locale -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since /** * Trait for calculating information gain. @@ -31,18 +31,15 @@ import org.apache.spark.annotation.{DeveloperApi, Since} trait Impurity extends Serializable { /** - * :: DeveloperApi :: * information calculation for multiclass classification * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels * @return information value, or 0 if totalCount = 0 */ @Since("1.1.0") - @DeveloperApi def calculate(counts: Array[Double], totalCount: Double): Double /** - * :: DeveloperApi :: * information calculation for regression * @param count number of instances * @param sum sum of labels @@ -50,7 +47,6 @@ trait Impurity extends Serializable { * @return information value, or 0 if count = 0 */ @Since("1.0.0") - @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index a07b919271f71..f5b2f8d514c7e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since /** * Class for calculating variance during regression @@ -26,27 +26,22 @@ import org.apache.spark.annotation.{DeveloperApi, Since} object Variance extends Impurity { /** - * :: DeveloperApi :: * information calculation for multiclass classification * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels * @return information value, or 0 if totalCount = 0 */ @Since("1.1.0") - @DeveloperApi override def calculate(counts: Array[Double], totalCount: Double): Double = throw new UnsupportedOperationException("Variance.calculate") /** - * :: DeveloperApi :: * variance calculation * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels * @return information value, or 0 if count = 0 */ - @Since("1.0.0") - @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = { if (count == 0) { return 0 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index 9b60d018d0eda..48e576afdf675 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since /** - * :: DeveloperApi :: * Class for absolute error loss calculation (for regression). * * The absolute (L1) error is defined as: @@ -29,7 +28,6 @@ import org.apache.spark.annotation.{DeveloperApi, Since} * where y is the label and F(x) is the model prediction for features x. */ @Since("1.2.0") -@DeveloperApi object AbsoluteError extends Loss { /** 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 9339f0a23c1bd..49f1215c77ad2 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 @@ -17,11 +17,10 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.util.MLUtils /** - * :: DeveloperApi :: * Class for log loss calculation (for classification). * This uses twice the binomial negative log likelihood, called "deviance" in Friedman (1999). * @@ -30,7 +29,6 @@ import org.apache.spark.mllib.util.MLUtils * where y is a label in {-1, 1} and F(x) is the model prediction for features x. */ @Since("1.2.0") -@DeveloperApi object LogLoss extends ClassificationLoss { /** 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 e7ffb3f8f53c1..f589e0ce5d4a1 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 @@ -17,17 +17,15 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since 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. */ @Since("1.2.0") -@DeveloperApi trait Loss extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index 4eb6810c46b20..3ff59fc64ce7f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since /** - * :: DeveloperApi :: * Class for squared error loss calculation. * * The squared (L2) error is defined as: @@ -29,7 +28,6 @@ import org.apache.spark.annotation.{DeveloperApi, Since} * where y is the label and F(x) is the model prediction for features x. */ @Since("1.2.0") -@DeveloperApi object SquaredError extends Loss { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index f3dbfd96e1815..54d95a3ffad1b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.tree.impurity.ImpurityCalculator /** - * :: DeveloperApi :: * Information gain statistics for each split * @param gain information gain value * @param impurity current node impurity @@ -31,7 +30,6 @@ import org.apache.spark.mllib.tree.impurity.ImpurityCalculator * @param rightPredict right node predict */ @Since("1.0.0") -@DeveloperApi class InformationGainStats( val gain: Double, val impurity: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 5fd053647aa46..bac36ce6937b0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -17,13 +17,12 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.tree.configuration.FeatureType._ /** - * :: DeveloperApi :: * Node in a decision tree. * * About node indexing: @@ -40,7 +39,6 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ * @param stats information gain stats */ @Since("1.0.0") -@DeveloperApi class Node @Since("1.2.0") ( @Since("1.0.0") val id: Int, @Since("1.0.0") var predict: Predict, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index 1dbdd2d860efd..44bc625d217f1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -17,16 +17,14 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since /** - * :: DeveloperApi :: * Predicted value for a node * @param predict predicted value * @param prob probability of the label (classification only) */ @Since("1.2.0") -@DeveloperApi class Predict @Since("1.2.0") ( @Since("1.2.0") val predict: Double, @Since("1.2.0") val prob: Double = 0.0) extends Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index bda5e662779c7..dbada8f983b99 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** - * :: DeveloperApi :: * Split applied to a feature * @param feature feature index * @param threshold Threshold for continuous feature. @@ -30,7 +29,6 @@ import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType * @param categories Split left if categorical feature value is in this set, else right. */ @Since("1.0.0") -@DeveloperApi case class Split( @Since("1.0.0") feature: Int, @Since("1.0.0") threshold: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index f7e8d98a10568..c5069277fad68 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -25,7 +25,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector @@ -174,7 +174,6 @@ class GradientBoostedTreesModel @Since("1.2.0") ( object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { /** - * :: DeveloperApi :: * Compute the initial predictions and errors for a dataset for the first * iteration of gradient boosting. * @param data: training data. @@ -185,7 +184,6 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { * corresponding to every sample. */ @Since("1.4.0") - @DeveloperApi def computeInitialPredictionAndError( data: RDD[LabeledPoint], initTreeWeight: Double, @@ -199,7 +197,6 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { } /** - * :: DeveloperApi :: * Update a zipped predictionError RDD * (as obtained with computeInitialPredictionAndError) * @param data: training data. @@ -211,7 +208,6 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { * corresponding to each sample. */ @Since("1.4.0") - @DeveloperApi def updatePredictionError( data: RDD[LabeledPoint], predictionAndError: RDD[(Double, Double)], diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 2c712d8f821a0..1a8c8807f91d4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -17,16 +17,14 @@ package org.apache.spark.mllib.util -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * A collection of methods used to validate data before applying ML algorithms. */ -@DeveloperApi @Since("0.8.0") object DataValidators extends Logging { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index b6eb10e9de00a..a409abc125c0c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -20,16 +20,14 @@ package org.apache.spark.mllib.util import scala.util.Random import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * Generate test data for KMeans. This class first chooses k cluster centers * from a d-dimensional Gaussian distribution scaled by factor r and then creates a Gaussian * cluster with scale 1 around each center. */ -@DeveloperApi @Since("0.8.0") object KMeansDataGenerator { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index de920da0a1f69..689ed0f842189 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -21,18 +21,16 @@ import scala.collection.JavaConverters._ import scala.util.Random import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{BLAS, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * Generate sample data used for Linear Data. This class generates * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the * response variable `Y`. */ -@DeveloperApi @Since("0.8.0") object LinearDataGenerator { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index 7e9d9465441c9..bca4ad4a3a60e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -20,17 +20,15 @@ package org.apache.spark.mllib.util import scala.util.Random import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * Generate test data for LogisticRegression. This class chooses positive labels * with probability `probOne` and scales features for positive examples by `eps`. */ -@DeveloperApi @Since("0.8.0") object LogisticRegressionDataGenerator { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 7a308a5ec25c0..de21db8d44d7c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -22,12 +22,11 @@ import java.{util => ju} import scala.util.Random import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix} import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * Generate RDD(s) containing data for Matrix Factorization. * * This method samples training entries according to the oversampling factor @@ -50,7 +49,6 @@ import org.apache.spark.rdd.RDD * test (Boolean) Whether to create testing RDD. * testSampFact (Double) Percentage of training data to use as test data. */ -@DeveloperApi @Since("0.8.0") object MFDataGenerator { @Since("0.8.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index 9f6ba025aedde..9fffa508afbfb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -22,17 +22,15 @@ import scala.util.Random import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: * Generate sample data used for SVM. This class generates uniform random values * for the features and adds Gaussian noise with weight 0.1 to generate labels. */ -@DeveloperApi @Since("0.8.0") object SVMDataGenerator { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala index e8889bfd7c381..8f2d8b9014c68 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala @@ -24,17 +24,14 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.types.{DataType, StructField, StructType} /** - * :: DeveloperApi :: - * * Trait for models and transformers which may be saved as files. * This should be inherited by the class which implements model instances. */ -@DeveloperApi @Since("1.3.0") trait Saveable { @@ -57,12 +54,9 @@ trait Saveable { } /** - * :: DeveloperApi :: - * * Trait for classes which can load models and transformers from files. * This should be inherited by an object paired with the model class. */ -@DeveloperApi @Since("1.3.0") trait Loader[M <: Saveable] { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ANOVASelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ANOVASelectorSuite.scala new file mode 100644 index 0000000000000..e07c212ca2d4c --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ANOVASelectorSuite.scala @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.sql.{Dataset, Row} + +class ANOVASelectorSuite extends MLTest with DefaultReadWriteTest { + + import testImplicits._ + + @transient var dataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + // scalastyle:off + /* + X: + array([[4.65415496e-03, 1.03550567e-01, -1.17358140e+00, + 1.61408773e-01, 3.92492111e-01, 7.31240882e-01], + [-9.01651741e-01, -5.28905302e-01, 1.27636785e+00, + 7.02154563e-01, 6.21348351e-01, 1.88397353e-01], + [ 3.85692159e-01, -9.04639637e-01, 5.09782604e-02, + 8.40043971e-01, 7.45977857e-01, 8.78402288e-01], + [ 1.36264353e+00, 2.62454094e-01, 7.96306202e-01, + 6.14948000e-01, 7.44948187e-01, 9.74034830e-01], + [ 9.65874070e-01, 2.52773665e+00, -2.19380094e+00, + 2.33408080e-01, 1.86340919e-01, 8.23390433e-01], + [ 1.12324305e+01, -2.77121515e-01, 1.12740513e-01, + 2.35184013e-01, 3.46668895e-01, 9.38500782e-02], + [ 1.06195839e+01, -1.82891238e+00, 2.25085601e-01, + 9.09979851e-01, 6.80257535e-02, 8.24017480e-01], + [ 1.12806837e+01, 1.30686889e+00, 9.32839108e-02, + 3.49784755e-01, 1.71322408e-02, 7.48465194e-02], + [ 9.98689462e+00, 9.50808938e-01, -2.90786359e-01, + 2.31253009e-01, 7.46270968e-01, 1.60308169e-01], + [ 1.08428551e+01, -1.02749936e+00, 1.73951508e-01, + 8.92482744e-02, 1.42651730e-01, 7.66751625e-01], + [-1.98641448e+00, 1.12811990e+01, -2.35246756e-01, + 8.22809049e-01, 3.26739456e-01, 7.88268404e-01], + [-6.09864090e-01, 1.07346276e+01, -2.18805509e-01, + 7.33931213e-01, 1.42554396e-01, 7.11225605e-01], + [-1.58481268e+00, 9.19364039e+00, -5.87490459e-02, + 2.51532056e-01, 2.82729807e-01, 7.16245686e-01], + [-2.50949277e-01, 1.12815254e+01, -6.94806734e-01, + 5.93898886e-01, 5.68425656e-01, 8.49762330e-01], + [ 7.63485129e-01, 1.02605138e+01, 1.32617719e+00, + 5.49682879e-01, 8.59931442e-01, 4.88677978e-02], + [ 9.34900015e-01, 4.11379043e-01, 8.65010205e+00, + 9.23509168e-01, 1.16995043e-01, 5.91894106e-03], + [ 4.73734933e-01, -1.48321181e+00, 9.73349621e+00, + 4.09421563e-01, 5.09375719e-01, 5.93157850e-01], + [ 3.41470679e-01, -6.88972582e-01, 9.60347938e+00, + 3.62654055e-01, 2.43437468e-01, 7.13052838e-01], + [-5.29614251e-01, -1.39262856e+00, 1.01354144e+01, + 8.24123861e-01, 5.84074506e-01, 6.54461558e-01], + [-2.99454508e-01, 2.20457263e+00, 1.14586015e+01, + 5.16336729e-01, 9.99776159e-01, 3.15769738e-01]]) + y: + array([1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3, 4, 4, 4, 4, 4]) + scikit-learn result: + >>> f_classif(X, y) + (array([228.27701422, 84.33070501, 134.25330675, 0.82211775, 0.82991363, 1.08478943]), + array([2.43864448e-13, 5.09088367e-10, 1.49033067e-11, 5.00596446e-01, 4.96684374e-01, 3.83798191e-01])) + */ + // scalastyle:on + + val data = Seq( + (1, Vectors.dense(4.65415496e-03, 1.03550567e-01, -1.17358140e+00, + 1.61408773e-01, 3.92492111e-01, 7.31240882e-01), Vectors.dense(4.65415496e-03)), + (1, Vectors.dense(-9.01651741e-01, -5.28905302e-01, 1.27636785e+00, + 7.02154563e-01, 6.21348351e-01, 1.88397353e-01), Vectors.dense(-9.01651741e-01)), + (1, Vectors.dense(3.85692159e-01, -9.04639637e-01, 5.09782604e-02, + 8.40043971e-01, 7.45977857e-01, 8.78402288e-01), Vectors.dense(3.85692159e-01)), + (1, Vectors.dense(1.36264353e+00, 2.62454094e-01, 7.96306202e-01, + 6.14948000e-01, 7.44948187e-01, 9.74034830e-01), Vectors.dense(1.36264353e+00)), + (1, Vectors.dense(9.65874070e-01, 2.52773665e+00, -2.19380094e+00, + 2.33408080e-01, 1.86340919e-01, 8.23390433e-01), Vectors.dense(9.65874070e-01)), + (2, Vectors.dense(1.12324305e+01, -2.77121515e-01, 1.12740513e-01, + 2.35184013e-01, 3.46668895e-01, 9.38500782e-02), Vectors.dense(1.12324305e+01)), + (2, Vectors.dense(1.06195839e+01, -1.82891238e+00, 2.25085601e-01, + 9.09979851e-01, 6.80257535e-02, 8.24017480e-01), Vectors.dense(1.06195839e+01)), + (2, Vectors.dense(1.12806837e+01, 1.30686889e+00, 9.32839108e-02, + 3.49784755e-01, 1.71322408e-02, 7.48465194e-02), Vectors.dense(1.12806837e+01)), + (2, Vectors.dense(9.98689462e+00, 9.50808938e-01, -2.90786359e-01, + 2.31253009e-01, 7.46270968e-01, 1.60308169e-01), Vectors.dense(9.98689462e+00)), + (2, Vectors.dense(1.08428551e+01, -1.02749936e+00, 1.73951508e-01, + 8.92482744e-02, 1.42651730e-01, 7.66751625e-01), Vectors.dense(1.08428551e+01)), + (3, Vectors.dense(-1.98641448e+00, 1.12811990e+01, -2.35246756e-01, + 8.22809049e-01, 3.26739456e-01, 7.88268404e-01), Vectors.dense(-1.98641448e+00)), + (3, Vectors.dense(-6.09864090e-01, 1.07346276e+01, -2.18805509e-01, + 7.33931213e-01, 1.42554396e-01, 7.11225605e-01), Vectors.dense(-6.09864090e-01)), + (3, Vectors.dense(-1.58481268e+00, 9.19364039e+00, -5.87490459e-02, + 2.51532056e-01, 2.82729807e-01, 7.16245686e-01), Vectors.dense(-1.58481268e+00)), + (3, Vectors.dense(-2.50949277e-01, 1.12815254e+01, -6.94806734e-01, + 5.93898886e-01, 5.68425656e-01, 8.49762330e-01), Vectors.dense(-2.50949277e-01)), + (3, Vectors.dense(7.63485129e-01, 1.02605138e+01, 1.32617719e+00, + 5.49682879e-01, 8.59931442e-01, 4.88677978e-02), Vectors.dense(7.63485129e-01)), + (4, Vectors.dense(9.34900015e-01, 4.11379043e-01, 8.65010205e+00, + 9.23509168e-01, 1.16995043e-01, 5.91894106e-03), Vectors.dense(9.34900015e-01)), + (4, Vectors.dense(4.73734933e-01, -1.48321181e+00, 9.73349621e+00, + 4.09421563e-01, 5.09375719e-01, 5.93157850e-01), Vectors.dense(4.73734933e-01)), + (4, Vectors.dense(3.41470679e-01, -6.88972582e-01, 9.60347938e+00, + 3.62654055e-01, 2.43437468e-01, 7.13052838e-01), Vectors.dense(3.41470679e-01)), + (4, Vectors.dense(-5.29614251e-01, -1.39262856e+00, 1.01354144e+01, + 8.24123861e-01, 5.84074506e-01, 6.54461558e-01), Vectors.dense(-5.29614251e-01)), + (4, Vectors.dense(-2.99454508e-01, 2.20457263e+00, 1.14586015e+01, + 5.16336729e-01, 9.99776159e-01, 3.15769738e-01), Vectors.dense(-2.99454508e-01))) + + dataset = spark.createDataFrame(data).toDF("label", "features", "topFeature") + } + + test("params") { + ParamsSuite.checkParams(new ANOVASelector()) + } + + test("Test ANOVAFValue calssification selector: numTopFeatures") { + val selector = new ANOVASelector() + .setOutputCol("filtered").setSelectorType("numTopFeatures").setNumTopFeatures(1) + testSelector(selector, dataset) + } + + test("Test ANOVAFValue calssification selector: percentile") { + val selector = new ANOVASelector() + .setOutputCol("filtered").setSelectorType("percentile").setPercentile(0.17) + testSelector(selector, dataset) + } + + test("Test ANOVAFValue calssification selector: fpr") { + val selector = new ANOVASelector() + .setOutputCol("filtered").setSelectorType("fpr").setFpr(1.0E-12) + testSelector(selector, dataset) + } + + test("Test ANOVAFValue calssification selector: fdr") { + val selector = new ANOVASelector() + .setOutputCol("filtered").setSelectorType("fdr").setFdr(6.0E-12) + testSelector(selector, dataset) + } + + test("Test ANOVAFValue calssification selector: fwe") { + val selector = new ANOVASelector() + .setOutputCol("filtered").setSelectorType("fwe").setFwe(6.0E-12) + testSelector(selector, dataset) + } + + test("read/write") { + def checkModelData(model: ANOVASelectorModel, model2: ANOVASelectorModel): Unit = { + assert(model.selectedFeatures === model2.selectedFeatures) + } + val anovaSelector = new ANOVASelector() + testEstimatorAndModelReadWrite(anovaSelector, dataset, + ANOVASelectorSuite.allParamSettings, + ANOVASelectorSuite.allParamSettings, checkModelData) + } + + private def testSelector(selector: ANOVASelector, data: Dataset[_]): + ANOVASelectorModel = { + val selectorModel = selector.fit(data) + testTransformer[(Double, Vector, Vector)](data.toDF(), selectorModel, + "filtered", "topFeature") { + case Row(vec1: Vector, vec2: Vector) => + assert(vec1 ~== vec2 absTol 1e-1) + } + selectorModel + } +} + +object ANOVASelectorSuite { + + /** + * 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. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "selectorType" -> "percentile", + "numTopFeatures" -> 1, + "percentile" -> 0.12, + "outputCol" -> "myOutput" + ) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/FValueSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/FValueSelectorSuite.scala index ace1ba41727d8..b54d6d351653f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/FValueSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/FValueSelectorSuite.scala @@ -34,88 +34,112 @@ class FValueSelectorSuite extends MLTest with DefaultReadWriteTest { // scalastyle:off /* - FValue REGRESSION - X (features) = - [[1.67318514e-01, 1.78398028e-01, 4.36846538e-01, 5.24003164e-01, 1.80915415e-01, 1.98030859e-01], - [3.71836586e-01, 6.13453963e-01, 7.15269190e-01, 9.33623792e-03, 5.36095674e-01, 2.74223333e-01], - [3.68988949e-01, 5.34104018e-01, 5.24858744e-01, 6.86815853e-01, 3.26534757e-01, 6.92699400e-01], - [4.87748505e-02, 3.07080315e-01, 7.82955385e-01, 6.90167375e-01, 6.44077919e-01, 4.23739024e-01], - [6.50153455e-01, 8.32746110e-01, 6.88029140e-03, 1.27859556e-01, 6.80223767e-01, 6.25825675e-01], - - [9.47343271e-01, 2.13193978e-01, 3.71342472e-01, 8.21291956e-01, 4.38195693e-01, 5.76569439e-01], - [9.96499254e-01, 8.45833297e-01, 6.56086922e-02, 5.90029174e-01, 1.68954572e-01, 7.19792823e-02], - [1.85926914e-01, 9.60329804e-01, 3.13487406e-01, 9.59549928e-01, 6.89093311e-01, 6.94999427e-01], - [9.40164576e-01, 2.69042714e-02, 5.39491321e-01, 5.74068666e-01, 1.10935343e-01, 2.17519760e-01], - [2.97951848e-02, 1.06592106e-01, 5.74931856e-01, 8.80801522e-01, 8.60445070e-01, 9.22757966e-01], - - [9.80970473e-01, 3.05909353e-01, 4.96401766e-01, 2.44342697e-01, 6.90559227e-01, 5.64858704e-01], - [1.55939260e-01, 2.18626853e-01, 5.01834270e-01, 1.86694987e-01, 9.15411148e-01, 6.40527848e-01], - [3.16107608e-01, 9.25906358e-01, 5.47327167e-01, 4.83712979e-01, 8.42305220e-01, 7.58488462e-01], - [4.14393503e-01, 1.30817883e-01, 5.62034942e-01, 1.05150633e-01, 5.35632795e-01, 9.47594074e-04], - [5.26233981e-01, 7.63781419e-02, 3.19188240e-01, 5.16528633e-02, 5.28416724e-01, 6.47050470e-03], - - [2.73404764e-01, 7.17070744e-01, 3.12889595e-01, 8.39271965e-01, 9.67650889e-01, 8.50098873e-01], - [4.63289495e-01, 3.57055416e-02, 5.43528596e-01, 4.44840919e-01, 9.36845855e-02, 7.81595037e-01], - [3.21784993e-01, 3.15622454e-01, 7.58870408e-01, 5.18198558e-01, 2.28151905e-01, 4.42460325e-01], - [3.72428352e-01, 1.44447969e-01, 8.40274188e-01, 5.86308041e-01, 6.09893953e-01, 3.97006473e-01], - [3.12776786e-01, 9.33630195e-01, 2.29328749e-01, 4.32807208e-01, 1.51703470e-02, 1.51589320e-01]] - - y (labels) = - [0.33997803, 0.71456716, 0.58676766, 0.52894227, 0.53158463, - 0.55515181, 0.67008744, 0.5966537 , 0.56255674, 0.33904133, - 0.66485577, 0.38514965, 0.73885841, 0.45766267, 0.34801557, - 0.52529452, 0.42503336, 0.60221968, 0.58964479, 0.58194949] - - Note that y = X @ w, where w = [0.3, 0.4, 0.5, 0. , 0. , 0. ] - - Sklearn results: - F values per feature: [2.76445780e+00, 1.05267800e+01, 4.43399092e-02, 2.04580501e-02, - 3.13208557e-02, 1.35248025e-03] - p values per feature: [0.11369388, 0.0044996 , 0.83558782, 0.88785417, 0.86150261, 0.97106833] + Use the following sklearn data in this test + + >>> from sklearn.feature_selection import f_regression + >>> import numpy as np + >>> np.random.seed(777) + >>> X = np.random.rand(20, 6) + >>> w = np.array([0.3, 0.4, 0.5, 0, 0, 0]) + >>> y = X @ w + >>> X + array([[0.19151945, 0.62210877, 0.43772774, 0.78535858, 0.77997581, + 0.27259261], + [0.27646426, 0.80187218, 0.95813935, 0.87593263, 0.35781727, + 0.50099513], + [0.68346294, 0.71270203, 0.37025075, 0.56119619, 0.50308317, + 0.01376845], + [0.77282662, 0.88264119, 0.36488598, 0.61539618, 0.07538124, + 0.36882401], + [0.9331401 , 0.65137814, 0.39720258, 0.78873014, 0.31683612, + 0.56809865], + [0.86912739, 0.43617342, 0.80214764, 0.14376682, 0.70426097, + 0.70458131], + [0.21879211, 0.92486763, 0.44214076, 0.90931596, 0.05980922, + 0.18428708], + [0.04735528, 0.67488094, 0.59462478, 0.53331016, 0.04332406, + 0.56143308], + [0.32966845, 0.50296683, 0.11189432, 0.60719371, 0.56594464, + 0.00676406], + [0.61744171, 0.91212289, 0.79052413, 0.99208147, 0.95880176, + 0.79196414], + [0.28525096, 0.62491671, 0.4780938 , 0.19567518, 0.38231745, + 0.05387369], + [0.45164841, 0.98200474, 0.1239427 , 0.1193809 , 0.73852306, + 0.58730363], + [0.47163253, 0.10712682, 0.22921857, 0.89996519, 0.41675354, + 0.53585166], + [0.00620852, 0.30064171, 0.43689317, 0.612149 , 0.91819808, + 0.62573667], + [0.70599757, 0.14983372, 0.74606341, 0.83100699, 0.63372577, + 0.43830988], + [0.15257277, 0.56840962, 0.52822428, 0.95142876, 0.48035918, + 0.50255956], + [0.53687819, 0.81920207, 0.05711564, 0.66942174, 0.76711663, + 0.70811536], + [0.79686718, 0.55776083, 0.96583653, 0.1471569 , 0.029647 , + 0.59389349], + [0.1140657 , 0.95080985, 0.32570741, 0.19361869, 0.45781165, + 0.92040257], + [0.87906916, 0.25261576, 0.34800879, 0.18258873, 0.90179605, + 0.70652816]]) + >>> y + array([0.52516321, 0.88275782, 0.67524507, 0.76734745, 0.73909458, + 0.83628141, 0.65665506, 0.58147135, 0.35603443, 0.94534373, + 0.57458887, 0.59026777, 0.29894977, 0.34056582, 0.64476446, + 0.53724782, 0.5173021 , 0.94508275, 0.57739736, 0.53877145]) + >>> f_regression(X, y) + (array([5.58025504, 3.98311705, 20.59605518, 0.07993376, 1.25127646, + 0.7676937 ]), + array([2.96302196e-02, 6.13173918e-02, 2.54580618e-04, 7.80612726e-01, + 2.78015517e-01, 3.92474567e-01])) */ // scalastyle:on val data = Seq( - (0.33997803, Vectors.dense(1.67318514e-01, 1.78398028e-01, 4.36846538e-01, - 5.24003164e-01, 1.80915415e-01, 1.98030859e-01), Vectors.dense(1.78398028e-01)), - (0.71456716, Vectors.dense(3.71836586e-01, 6.13453963e-01, 7.15269190e-01, - 9.33623792e-03, 5.36095674e-01, 2.74223333e-01), Vectors.dense(6.13453963e-01)), - (0.58676766, Vectors.dense(3.68988949e-01, 5.34104018e-01, 5.24858744e-01, - 6.86815853e-01, 3.26534757e-01, 6.92699400e-01), Vectors.dense(5.34104018e-01)), - (0.52894227, Vectors.dense(4.87748505e-02, 3.07080315e-01, 7.82955385e-01, - 6.90167375e-01, 6.44077919e-01, 4.23739024e-01), Vectors.dense(3.07080315e-01)), - (0.53158463, Vectors.dense(6.50153455e-01, 8.32746110e-01, 6.88029140e-03, - 1.27859556e-01, 6.80223767e-01, 6.25825675e-01), Vectors.dense(8.32746110e-01)), - (0.55515181, Vectors.dense(9.47343271e-01, 2.13193978e-01, 3.71342472e-01, - 8.21291956e-01, 4.38195693e-01, 5.76569439e-01), Vectors.dense(2.13193978e-01)), - (0.67008744, Vectors.dense(9.96499254e-01, 8.45833297e-01, 6.56086922e-02, - 5.90029174e-01, 1.68954572e-01, 7.19792823e-02), Vectors.dense(8.45833297e-01)), - (0.5966537, Vectors.dense(1.85926914e-01, 9.60329804e-01, 3.13487406e-01, - 9.59549928e-01, 6.89093311e-01, 6.94999427e-01), Vectors.dense(9.60329804e-01)), - (0.56255674, Vectors.dense(9.40164576e-01, 2.69042714e-02, 5.39491321e-01, - 5.74068666e-01, 1.10935343e-01, 2.17519760e-01), Vectors.dense(2.69042714e-02)), - (0.33904133, Vectors.dense(2.97951848e-02, 1.06592106e-01, 5.74931856e-01, - 8.80801522e-01, 8.60445070e-01, 9.22757966e-01), Vectors.dense(1.06592106e-01)), - (0.66485577, Vectors.dense(9.80970473e-01, 3.05909353e-01, 4.96401766e-01, - 2.44342697e-01, 6.90559227e-01, 5.64858704e-01), Vectors.dense(3.05909353e-01)), - (0.38514965, Vectors.dense(1.55939260e-01, 2.18626853e-01, 5.01834270e-01, - 1.86694987e-01, 9.15411148e-01, 6.40527848e-01), Vectors.dense(2.18626853e-01)), - (0.73885841, Vectors.dense(3.16107608e-01, 9.25906358e-01, 5.47327167e-01, - 4.83712979e-01, 8.42305220e-01, 7.58488462e-01), Vectors.dense(9.25906358e-01)), - (0.45766267, Vectors.dense(4.14393503e-01, 1.30817883e-01, 5.62034942e-01, - 1.05150633e-01, 5.35632795e-01, 9.47594074e-04), Vectors.dense(1.30817883e-01)), - (0.34801557, Vectors.dense(5.26233981e-01, 7.63781419e-02, 3.19188240e-01, - 5.16528633e-02, 5.28416724e-01, 6.47050470e-03), Vectors.dense(7.63781419e-02)), - (0.52529452, Vectors.dense(2.73404764e-01, 7.17070744e-01, 3.12889595e-01, - 8.39271965e-01, 9.67650889e-01, 8.50098873e-01), Vectors.dense(7.17070744e-01)), - (0.42503336, Vectors.dense(4.63289495e-01, 3.57055416e-02, 5.43528596e-01, - 4.44840919e-01, 9.36845855e-02, 7.81595037e-01), Vectors.dense(3.57055416e-02)), - (0.60221968, Vectors.dense(3.21784993e-01, 3.15622454e-01, 7.58870408e-01, - 5.18198558e-01, 2.28151905e-01, 4.42460325e-01), Vectors.dense(3.15622454e-01)), - (0.58964479, Vectors.dense(3.72428352e-01, 1.44447969e-01, 8.40274188e-01, - 5.86308041e-01, 6.09893953e-01, 3.97006473e-01), Vectors.dense(1.44447969e-01)), - (0.58194949, Vectors.dense(3.12776786e-01, 9.33630195e-01, 2.29328749e-01, - 4.32807208e-01, 1.51703470e-02, 1.51589320e-01), Vectors.dense(9.33630195e-01))) + (0.52516321, Vectors.dense(0.19151945, 0.62210877, 0.43772774, 0.78535858, 0.77997581, + 0.27259261), Vectors.dense(0.43772774)), + (0.88275782, Vectors.dense(0.27646426, 0.80187218, 0.95813935, 0.87593263, 0.35781727, + 0.50099513), Vectors.dense(0.95813935)), + (0.67524507, Vectors.dense(0.68346294, 0.71270203, 0.37025075, 0.56119619, 0.50308317, + 0.01376845), Vectors.dense(0.37025075)), + (0.76734745, Vectors.dense(0.77282662, 0.88264119, 0.36488598, 0.61539618, 0.07538124, + 0.36882401), Vectors.dense(0.36488598)), + (0.73909458, Vectors.dense(0.9331401, 0.65137814, 0.39720258, 0.78873014, 0.31683612, + 0.56809865), Vectors.dense(0.39720258)), + + (0.83628141, Vectors.dense(0.86912739, 0.43617342, 0.80214764, 0.14376682, 0.70426097, + 0.70458131), Vectors.dense(0.80214764)), + (0.65665506, Vectors.dense(0.21879211, 0.92486763, 0.44214076, 0.90931596, 0.05980922, + 0.18428708), Vectors.dense(0.44214076)), + (0.58147135, Vectors.dense(0.04735528, 0.67488094, 0.59462478, 0.53331016, 0.04332406, + 0.56143308), Vectors.dense(0.59462478)), + (0.35603443, Vectors.dense(0.32966845, 0.50296683, 0.11189432, 0.60719371, 0.56594464, + 0.00676406), Vectors.dense(0.11189432)), + (0.94534373, Vectors.dense(0.61744171, 0.91212289, 0.79052413, 0.99208147, 0.95880176, + 0.79196414), Vectors.dense(0.79052413)), + + (0.57458887, Vectors.dense(0.28525096, 0.62491671, 0.4780938, 0.19567518, 0.38231745, + 0.05387369), Vectors.dense(0.4780938)), + (0.59026777, Vectors.dense(0.45164841, 0.98200474, 0.1239427, 0.1193809, 0.73852306, + 0.58730363), Vectors.dense(0.1239427)), + (0.29894977, Vectors.dense(0.47163253, 0.10712682, 0.22921857, 0.89996519, 0.41675354, + 0.53585166), Vectors.dense(0.22921857)), + (0.34056582, Vectors.dense(0.00620852, 0.30064171, 0.43689317, 0.612149, 0.91819808, + 0.62573667), Vectors.dense(0.43689317)), + (0.64476446, Vectors.dense(0.70599757, 0.14983372, 0.74606341, 0.83100699, 0.63372577, + 0.43830988), Vectors.dense(0.74606341)), + + (0.53724782, Vectors.dense(0.15257277, 0.56840962, 0.52822428, 0.95142876, 0.48035918, + 0.50255956), Vectors.dense(0.52822428)), + (0.5173021, Vectors.dense(0.53687819, 0.81920207, 0.05711564, 0.66942174, 0.76711663, + 0.70811536), Vectors.dense(0.05711564)), + (0.94508275, Vectors.dense(0.79686718, 0.55776083, 0.96583653, 0.1471569, 0.029647, + 0.59389349), Vectors.dense(0.96583653)), + (0.57739736, Vectors.dense(0.1140657, 0.95080985, 0.96583653, 0.19361869, 0.45781165, + 0.92040257), Vectors.dense(0.96583653)), + (0.53877145, Vectors.dense(0.87906916, 0.25261576, 0.34800879, 0.18258873, 0.90179605, + 0.70652816), Vectors.dense(0.34800879))) dataset = spark.createDataFrame(data).toDF("label", "features", "topFeature") } @@ -127,21 +151,18 @@ class FValueSelectorSuite extends MLTest with DefaultReadWriteTest { test("Test FValue selector: numTopFeatures") { val selector = new FValueSelector() .setOutputCol("filtered").setSelectorType("numTopFeatures").setNumTopFeatures(1) - val model = selector.fit(dataset) testSelector(selector, dataset) } test("Test F Value selector: percentile") { val selector = new FValueSelector() .setOutputCol("filtered").setSelectorType("percentile").setPercentile(0.17) - val model = selector.fit(dataset) testSelector(selector, dataset) } test("Test F Value selector: fpr") { val selector = new FValueSelector() .setOutputCol("filtered").setSelectorType("fpr").setFpr(0.01) - val model = selector.fit(dataset) testSelector(selector, dataset) } @@ -169,7 +190,6 @@ class FValueSelectorSuite extends MLTest with DefaultReadWriteTest { val selector = new FValueSelector() .setOutputCol("filtered").setSelectorType("numTopFeatures").setNumTopFeatures(1) - val model = selector.fit(df) testSelector(selector, df) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VarianceThresholdSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VarianceThresholdSelectorSuite.scala new file mode 100644 index 0000000000000..6cc980397f9e9 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VarianceThresholdSelectorSuite.scala @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.sql.{Dataset, Row} + +class VarianceThresholdSelectorSuite extends MLTest with DefaultReadWriteTest { + + import testImplicits._ + + @transient var dataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val data = Seq( + (1, Vectors.dense(Array(6.0, 7.0, 0.0, 5.0, 6.0, 0.0)), + Vectors.dense(Array(6.0, 7.0, 0.0, 6.0, 0.0))), + (2, Vectors.dense(Array(0.0, 9.0, 6.0, 5.0, 5.0, 9.0)), + Vectors.dense(Array(0.0, 9.0, 6.0, 5.0, 9.0))), + (3, Vectors.dense(Array(0.0, 9.0, 3.0, 5.0, 5.0, 5.0)), + Vectors.dense(Array(0.0, 9.0, 3.0, 5.0, 5.0))), + (4, Vectors.dense(Array(0.0, 9.0, 8.0, 5.0, 6.0, 4.0)), + Vectors.dense(Array(0.0, 9.0, 8.0, 6.0, 4.0))), + (5, Vectors.dense(Array(8.0, 9.0, 6.0, 5.0, 4.0, 4.0)), + Vectors.dense(Array(8.0, 9.0, 6.0, 4.0, 4.0))), + (6, Vectors.dense(Array(8.0, 9.0, 6.0, 5.0, 0.0, 0.0)), + Vectors.dense(Array(8.0, 9.0, 6.0, 0.0, 0.0)))) + + dataset = spark.createDataFrame(data).toDF("id", "features", "expected") + } + + test("params") { + ParamsSuite.checkParams(new VarianceThresholdSelector) + } + + test("Test VarianceThresholdSelector: varainceThreshold not set") { + val selector = new VarianceThresholdSelector().setOutputCol("filtered") + testSelector(selector, dataset) + } + + test("Test VarianceThresholdSelector: set varianceThreshold") { + val df = spark.createDataFrame(Seq( + (1, Vectors.dense(Array(6.0, 7.0, 0.0, 7.0, 6.0, 0.0)), + Vectors.dense(Array(6.0, 7.0, 0.0))), + (2, Vectors.dense(Array(0.0, 9.0, 6.0, 0.0, 5.0, 9.0)), + Vectors.dense(Array(0.0, 0.0, 9.0))), + (3, Vectors.dense(Array(0.0, 9.0, 3.0, 0.0, 5.0, 5.0)), + Vectors.dense(Array(0.0, 0.0, 5.0))), + (4, Vectors.dense(Array(0.0, 9.0, 8.0, 5.0, 6.0, 4.0)), + Vectors.dense(Array(0.0, 5.0, 4.0))), + (5, Vectors.dense(Array(8.0, 9.0, 6.0, 5.0, 4.0, 4.0)), + Vectors.dense(Array(8.0, 5.0, 4.0))), + (6, Vectors.dense(Array(8.0, 9.0, 6.0, 0.0, 0.0, 0.0)), + Vectors.dense(Array(8.0, 0.0, 0.0))) + )).toDF("id", "features", "expected") + val selector = new VarianceThresholdSelector() + .setVarianceThreshold(8.2) + .setOutputCol("filtered") + testSelector(selector, df) + } + + test("Test VarianceThresholdSelector: sparse vector") { + val df = spark.createDataFrame(Seq( + (1, Vectors.sparse(6, Array((0, 6.0), (1, 7.0), (3, 7.0), (4, 6.0))), + Vectors.dense(Array(6.0, 0.0, 7.0, 0.0))), + (2, Vectors.sparse(6, Array((1, 9.0), (2, 6.0), (4, 5.0), (5, 9.0))), + Vectors.dense(Array(0.0, 6.0, 0.0, 9.0))), + (3, Vectors.sparse(6, Array((1, 9.0), (2, 3.0), (4, 5.0), (5, 5.0))), + Vectors.dense(Array(0.0, 3.0, 0.0, 5.0))), + (4, Vectors.dense(Array(0.0, 9.0, 8.0, 5.0, 6.0, 4.0)), + Vectors.dense(Array(0.0, 8.0, 5.0, 4.0))), + (5, Vectors.dense(Array(8.0, 9.0, 6.0, 5.0, 4.0, 4.0)), + Vectors.dense(Array(8.0, 6.0, 5.0, 4.0))), + (6, Vectors.dense(Array(8.0, 9.0, 6.0, 4.0, 0.0, 0.0)), + Vectors.dense(Array(8.0, 6.0, 4.0, 0.0))) + )).toDF("id", "features", "expected") + val selector = new VarianceThresholdSelector() + .setVarianceThreshold(8.1) + .setOutputCol("filtered") + testSelector(selector, df) + } + + test("read/write") { + def checkModelData(model: VarianceThresholdSelectorModel, model2: + VarianceThresholdSelectorModel): Unit = { + assert(model.selectedFeatures === model2.selectedFeatures) + } + val varSelector = new VarianceThresholdSelector + testEstimatorAndModelReadWrite(varSelector, dataset, + VarianceThresholdSelectorSuite.allParamSettings, + VarianceThresholdSelectorSuite.allParamSettings, checkModelData) + } + + private def testSelector(selector: VarianceThresholdSelector, data: Dataset[_]): + VarianceThresholdSelectorModel = { + val selectorModel = selector.fit(data) + testTransformer[(Int, Vector, Vector)](data.toDF(), selectorModel, + "filtered", "expected") { + case Row(vec1: Vector, vec2: Vector) => + assert(vec1 ~== vec2 absTol 1e-6) + } + selectorModel + } +} + +object VarianceThresholdSelectorSuite { + + /** + * 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. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "varianceThreshold" -> 0.12, + "outputCol" -> "myOutput" + ) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index a4d1d453ca5c1..9029fc96b36a8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -984,49 +984,27 @@ class ALSCleanerSuite extends SparkFunSuite with BeforeAndAfterEach { super.afterEach() } - test("ALS shuffle cleanup standalone") { - val conf = new SparkConf() - val localDir = Utils.createTempDir() - val checkpointDir = Utils.createTempDir() - def getAllFiles: Set[File] = - FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet - try { - conf.set("spark.local.dir", localDir.getAbsolutePath) - val sc = new SparkContext("local[2]", "test", conf) - try { - sc.setCheckpointDir(checkpointDir.getAbsolutePath) - // Test checkpoint and clean parents - val input = sc.parallelize(1 to 1000) - val keyed = input.map(x => (x % 20, 1)) - val shuffled = keyed.reduceByKey(_ + _) - val keysOnly = shuffled.keys - val deps = keysOnly.dependencies - keysOnly.count() - ALS.cleanShuffleDependencies(sc, deps, true) - val resultingFiles = getAllFiles - assert(resultingFiles === Set()) - // Ensure running count again works fine even if we kill the shuffle files. - keysOnly.count() - } finally { - sc.stop() - } - } finally { - Utils.deleteRecursively(localDir) - Utils.deleteRecursively(checkpointDir) - } - } - test("ALS shuffle cleanup in algorithm") { val conf = new SparkConf() val localDir = Utils.createTempDir() val checkpointDir = Utils.createTempDir() - def getAllFiles: Set[File] = - FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + def getAllFiles: Set[File] = { + val files = FileUtils.listFiles( + localDir, + TrueFileFilter.INSTANCE, + TrueFileFilter.INSTANCE).asScala.toSet + files + } try { conf.set("spark.local.dir", localDir.getAbsolutePath) val sc = new SparkContext("local[2]", "ALSCleanerSuite", conf) + val pattern = "shuffle_(\\d+)_.+\\.data".r try { sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // There should be 0 shuffle files at the start + val initialIds = getAllFiles.flatMap { f => + pattern.findAllIn(f.getName()).matchData.map { _.group(1) } } + assert(initialIds.size === 0) // Generate test data val (training, _) = ALSSuite.genImplicitTestData(sc, 20, 5, 1, 0.2, 0) // Implicitly test the cleaning of parents during ALS training @@ -1044,7 +1022,6 @@ class ALSCleanerSuite extends SparkFunSuite with BeforeAndAfterEach { val resultingFiles = getAllFiles // We expect the last shuffles files, block ratings, user factors, and item factors to be // around but no more. - val pattern = "shuffle_(\\d+)_.+\\.data".r val rddIds = resultingFiles.flatMap { f => pattern.findAllIn(f.getName()).matchData.map { _.group(1) } } assert(rddIds.size === 4) diff --git a/mllib/src/test/scala/org/apache/spark/ml/stat/ANOVATestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/stat/ANOVATestSuite.scala new file mode 100644 index 0000000000000..3f4a70d86b67f --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/stat/ANOVATestSuite.scala @@ -0,0 +1,174 @@ +/* + * 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.stat + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class ANOVATestSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + test("test DataFrame of labeled points") { + // scalastyle:off + /* + Use the following sklearn data in this test + + >>> from sklearn.feature_selection import f_classif + >>> import numpy as np + >>> np.random.seed(888) + >>> X = np.random.rand(20, 6) + >>> X + array([[0.85956061, 0.1645695 , 0.48347596, 0.92102727, 0.42855644, + 0.05746009], + [0.92500743, 0.65760154, 0.13295284, 0.53344893, 0.8994776 , + 0.24836496], + [0.03017182, 0.07244715, 0.87416449, 0.55843035, 0.91604736, + 0.63346045], + [0.28325261, 0.36536881, 0.09223386, 0.37251258, 0.34742278, + 0.70517077], + [0.64850904, 0.04090877, 0.21173176, 0.00148992, 0.13897166, + 0.21182539], + [0.02609493, 0.44608735, 0.23910531, 0.95449222, 0.90763182, + 0.8624905 ], + [0.09158744, 0.97745235, 0.41150139, 0.45830467, 0.52590925, + 0.29441554], + [0.97211594, 0.1814442 , 0.30340642, 0.17445413, 0.52756958, + 0.02069296], + [0.06354593, 0.63527231, 0.49620335, 0.0141264 , 0.62722219, + 0.63497507], + [0.10814149, 0.8296426 , 0.51775217, 0.57068344, 0.54633305, + 0.12714921], + [0.72731796, 0.94010124, 0.45007811, 0.87650674, 0.53735565, + 0.49568415], + [0.41827208, 0.85100628, 0.38685271, 0.60689503, 0.21784097, + 0.91294433], + [0.65843656, 0.5880859 , 0.18862706, 0.856398 , 0.18029327, + 0.94851926], + [0.3841634 , 0.25138793, 0.96746644, 0.77048045, 0.44685196, + 0.19813854], + [0.65982267, 0.23024125, 0.13598434, 0.60144265, 0.57848927, + 0.85623564], + [0.35764189, 0.47623815, 0.5459232 , 0.79508298, 0.14462443, + 0.01802919], + [0.38532153, 0.90614554, 0.86629571, 0.13988735, 0.32062385, + 0.00179492], + [0.2142368 , 0.28306022, 0.59481646, 0.42567028, 0.52207663, + 0.78082401], + [0.20788283, 0.76861782, 0.59595468, 0.62103642, 0.17781246, + 0.77655345], + [0.1751708 , 0.4547537 , 0.46187865, 0.79781199, 0.05104487, + 0.42406092]]) + >>> y = np.array([3, 2, 1, 5, 4, 4, 5, 4, 2, 1, 1, 2, 3, 4, 5, 1, 5, 3, 1, 1]) + >>> y + array([3, 2, 1, 5, 4, 4, 5, 4, 2, 1, 1, 2, 3, 4, 5, 1, 5, 3, 1, 1]) + >>> f_classif(X, y) + (array([0.64110932, 1.98689258, 0.55499714, 1.40340562, 0.30881722, + 0.3848595 ]), array([0.64137831, 0.14830724, 0.69858474, 0.28038169, 0.86759161, + 0.81608606])) + */ + // scalastyle:on + + val data = Seq( + LabeledPoint(3, Vectors.dense(0.85956061, 0.1645695, 0.48347596, 0.92102727, 0.42855644, + 0.05746009)), + LabeledPoint(2, Vectors.dense(0.92500743, 0.65760154, 0.13295284, 0.53344893, 0.8994776, + 0.24836496)), + LabeledPoint(1, Vectors.dense(0.03017182, 0.07244715, 0.87416449, 0.55843035, 0.91604736, + 0.63346045)), + LabeledPoint(5, Vectors.dense(0.28325261, 0.36536881, 0.09223386, 0.37251258, 0.34742278, + 0.70517077)), + LabeledPoint(4, Vectors.dense(0.64850904, 0.04090877, 0.21173176, 0.00148992, 0.13897166, + 0.21182539)), + LabeledPoint(4, Vectors.dense(0.02609493, 0.44608735, 0.23910531, 0.95449222, 0.90763182, + 0.8624905)), + LabeledPoint(5, Vectors.dense(0.09158744, 0.97745235, 0.41150139, 0.45830467, 0.52590925, + 0.29441554)), + LabeledPoint(4, Vectors.dense(0.97211594, 0.1814442, 0.30340642, 0.17445413, 0.52756958, + 0.02069296)), + LabeledPoint(2, Vectors.dense(0.06354593, 0.63527231, 0.49620335, 0.0141264, 0.62722219, + 0.63497507)), + LabeledPoint(1, Vectors.dense(0.10814149, 0.8296426, 0.51775217, 0.57068344, 0.54633305, + 0.12714921)), + LabeledPoint(1, Vectors.dense(0.72731796, 0.94010124, 0.45007811, 0.87650674, 0.53735565, + 0.49568415)), + LabeledPoint(2, Vectors.dense(0.41827208, 0.85100628, 0.38685271, 0.60689503, 0.21784097, + 0.91294433)), + LabeledPoint(3, Vectors.dense(0.65843656, 0.5880859, 0.18862706, 0.856398, 0.18029327, + 0.94851926)), + LabeledPoint(4, Vectors.dense(0.3841634, 0.25138793, 0.96746644, 0.77048045, 0.44685196, + 0.19813854)), + LabeledPoint(5, Vectors.dense(0.65982267, 0.23024125, 0.13598434, 0.60144265, 0.57848927, + 0.85623564)), + LabeledPoint(1, Vectors.dense(0.35764189, 0.47623815, 0.5459232, 0.79508298, 0.14462443, + 0.01802919)), + LabeledPoint(5, Vectors.dense(0.38532153, 0.90614554, 0.86629571, 0.13988735, 0.32062385, + 0.00179492)), + LabeledPoint(3, Vectors.dense(0.2142368, 0.28306022, 0.59481646, 0.42567028, 0.52207663, + 0.78082401)), + LabeledPoint(1, Vectors.dense(0.20788283, 0.76861782, 0.59595468, 0.62103642, 0.17781246, + 0.77655345)), + LabeledPoint(1, Vectors.dense(0.1751708, 0.4547537, 0.46187865, 0.79781199, 0.05104487, + 0.42406092))) + + for (numParts <- List(2, 4, 6, 8)) { + val df = spark.createDataFrame(sc.parallelize(data, numParts)) + val anovaResult = ANOVATest.test(df, "features", "label") + val (pValues: Vector, fValues: Vector) = + anovaResult.select("pValues", "fValues") + .as[(Vector, Vector)].head() + assert(pValues ~== Vectors.dense(0.64137831, 0.14830724, 0.69858474, 0.28038169, 0.86759161, + 0.81608606) relTol 1e-6) + assert(fValues ~== Vectors.dense(0.64110932, 1.98689258, 0.55499714, 1.40340562, 0.30881722, + 0.3848595) relTol 1e-6) + } + } + + test("test DataFrame with sparse vector") { + val data = Seq( + (3, Vectors.dense(Array(6.0, 7.0, 0.0, 7.0, 6.0, 0.0, 0.0))), + (1, Vectors.dense(Array(0.0, 9.0, 6.0, 0.0, 5.0, 9.0, 0.0))), + (3, Vectors.dense(Array(0.0, 9.0, 3.0, 0.0, 5.0, 5.0, 0.0))), + (2, Vectors.dense(Array(0.0, 9.0, 8.0, 5.0, 6.0, 4.0, 0.0))), + (2, Vectors.dense(Array(8.0, 9.0, 6.0, 5.0, 4.0, 4.0, 0.0))), + (3, Vectors.dense(Array(8.0, 9.0, 6.0, 4.0, 0.0, 0.0, 0.0)))) + + val df1 = spark.createDataFrame(data.map(t => (t._1, t._2.toDense))) + .toDF("label", "features") + val df2 = spark.createDataFrame(data.map(t => (t._1, t._2.toSparse))) + .toDF("label", "features") + val df3 = spark.createDataFrame(data.map(t => (t._1, t._2.compressed))) + .toDF("label", "features") + + Seq(df1, df2, df3).foreach { df => + val anovaResult = ANOVATest.test(df, "features", "label") + val (pValues: Vector, fValues: Vector) = + anovaResult.select("pValues", "fValues") + .as[(Vector, Vector)].head() + assert(pValues ~== Vectors.dense(0.71554175, 0.71554175, 0.34278574, 0.45824059, 0.84633632, + 0.15673368, Double.NaN) relTol 1e-6) + assert(fValues ~== Vectors.dense(0.375, 0.375, 1.5625, 1.02364865, 0.17647059, + 3.66, Double.NaN) relTol 1e-6) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/stat/ChiSquareTestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/stat/ChiSquareTestSuite.scala index 986c7f4c6ae57..bb34f3da23296 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/stat/ChiSquareTestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/stat/ChiSquareTestSuite.scala @@ -117,7 +117,7 @@ class ChiSquareTestSuite withClue("ChiSquare should throw an exception when given a continuous-valued label") { intercept[SparkException] { val df = spark.createDataFrame(continuousLabel) - ChiSquareTest.test(df, "features", "label") + ChiSquareTest.test(df, "features", "label").count() } } val continuousFeature = Seq.fill(tooManyCategories)( @@ -125,7 +125,7 @@ class ChiSquareTestSuite withClue("ChiSquare should throw an exception when given continuous-valued features") { intercept[SparkException] { val df = spark.createDataFrame(continuousFeature) - ChiSquareTest.test(df, "features", "label") + ChiSquareTest.test(df, "features", "label").count() } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/stat/FValueTestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/stat/FValueTestSuite.scala index 540c4fb0b63dc..37195d2b503bf 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/stat/FValueTestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/stat/FValueTestSuite.scala @@ -36,103 +36,104 @@ class FValueTestSuite >>> from sklearn.feature_selection import f_regression >>> import numpy as np + >>> np.random.seed(777) >>> X = np.random.rand(20, 6) >>> w = np.array([0.3, 0.4, 0.5, 0, 0, 0]) >>> y = X @ w >>> X - array([[1.67318514e-01, 1.78398028e-01, 4.36846538e-01, 5.24003164e-01, - 1.80915415e-01, 1.98030859e-01], - [3.71836586e-01, 6.13453963e-01, 7.15269190e-01, 9.33623792e-03, - 5.36095674e-01, 2.74223333e-01], - [3.68988949e-01, 5.34104018e-01, 5.24858744e-01, 6.86815853e-01, - 3.26534757e-01, 6.92699400e-01], - [4.87748505e-02, 3.07080315e-01, 7.82955385e-01, 6.90167375e-01, - 6.44077919e-01, 4.23739024e-01], - [6.50153455e-01, 8.32746110e-01, 6.88029140e-03, 1.27859556e-01, - 6.80223767e-01, 6.25825675e-01], - [9.47343271e-01, 2.13193978e-01, 3.71342472e-01, 8.21291956e-01, - 4.38195693e-01, 5.76569439e-01], - [9.96499254e-01, 8.45833297e-01, 6.56086922e-02, 5.90029174e-01, - 1.68954572e-01, 7.19792823e-02], - [1.85926914e-01, 9.60329804e-01, 3.13487406e-01, 9.59549928e-01, - 6.89093311e-01, 6.94999427e-01], - [9.40164576e-01, 2.69042714e-02, 5.39491321e-01, 5.74068666e-01, - 1.10935343e-01, 2.17519760e-01], - [2.97951848e-02, 1.06592106e-01, 5.74931856e-01, 8.80801522e-01, - 8.60445070e-01, 9.22757966e-01], - [9.80970473e-01, 3.05909353e-01, 4.96401766e-01, 2.44342697e-01, - 6.90559227e-01, 5.64858704e-01], - [1.55939260e-01, 2.18626853e-01, 5.01834270e-01, 1.86694987e-01, - 9.15411148e-01, 6.40527848e-01], - [3.16107608e-01, 9.25906358e-01, 5.47327167e-01, 4.83712979e-01, - 8.42305220e-01, 7.58488462e-01], - [4.14393503e-01, 1.30817883e-01, 5.62034942e-01, 1.05150633e-01, - 5.35632795e-01, 9.47594074e-04], - [5.26233981e-01, 7.63781419e-02, 3.19188240e-01, 5.16528633e-02, - 5.28416724e-01, 6.47050470e-03], - [2.73404764e-01, 7.17070744e-01, 3.12889595e-01, 8.39271965e-01, - 9.67650889e-01, 8.50098873e-01], - [4.63289495e-01, 3.57055416e-02, 5.43528596e-01, 4.44840919e-01, - 9.36845855e-02, 7.81595037e-01], - [3.21784993e-01, 3.15622454e-01, 7.58870408e-01, 5.18198558e-01, - 2.28151905e-01, 4.42460325e-01], - [3.72428352e-01, 1.44447969e-01, 8.40274188e-01, 5.86308041e-01, - 6.09893953e-01, 3.97006473e-01], - [3.12776786e-01, 9.33630195e-01, 2.29328749e-01, 4.32807208e-01, - 1.51703470e-02, 1.51589320e-01]]) + array([[0.15266373, 0.30235661, 0.06203641, 0.45986034, 0.83525338, + 0.92699705], + [0.72698898, 0.76849622, 0.26920507, 0.64402929, 0.09337326, + 0.07968589], + [0.58961375, 0.34334054, 0.98887615, 0.62647321, 0.68177928, + 0.55225681], + [0.26886006, 0.37325939, 0.2229281 , 0.1864426 , 0.39064809, + 0.19316241], + [0.61091093, 0.88280845, 0.62233882, 0.25311894, 0.17993031, + 0.81640447], + [0.22537162, 0.51685714, 0.51849582, 0.60037494, 0.53262048, + 0.01331005], + [0.52409726, 0.89588471, 0.76990129, 0.1228517 , 0.29587269, + 0.61202358], + [0.72613812, 0.46349747, 0.76911037, 0.19163103, 0.55786672, + 0.55077816], + [0.47222549, 0.79188496, 0.11524968, 0.6813039 , 0.36233361, + 0.34420889], + [0.44951875, 0.02694226, 0.41524769, 0.9222317 , 0.09120557, + 0.31512178], + [0.52802224, 0.32806203, 0.44891554, 0.01633442, 0.0970269 , + 0.69258857], + [0.83594341, 0.42432199, 0.8487743 , 0.54679121, 0.35410346, + 0.72724968], + [0.09385168, 0.8928588 , 0.33625828, 0.89183268, 0.296849 , + 0.30164829], + [0.80624061, 0.83760997, 0.63428133, 0.3113273 , 0.02944858, + 0.39977732], + [0.51817346, 0.00738845, 0.77494778, 0.8544712 , 0.13153282, + 0.28767364], + [0.32658881, 0.90655956, 0.99955954, 0.77088429, 0.04284752, + 0.96525111], + [0.97521246, 0.2025168 , 0.67985305, 0.46534506, 0.92001748, + 0.72820735], + [0.24585653, 0.01953996, 0.70598881, 0.77448287, 0.4729746 , + 0.80146736], + [0.17539792, 0.72016934, 0.3678759 , 0.53209295, 0.29719397, + 0.37429151], + [0.72810013, 0.39850784, 0.1058295 , 0.39858265, 0.52196395, + 0.1060125 ]]) >>> y - array([0.33997803, 0.71456716, 0.58676766, 0.52894227, 0.53158463, - 0.55515181, 0.67008744, 0.5966537 , 0.56255674, 0.33904133, - 0.66485577, 0.38514965, 0.73885841, 0.45766267, 0.34801557, - 0.52529452, 0.42503336, 0.60221968, 0.58964479, 0.58194949]) + array([0.19775997, 0.66009772, 0.80865842, 0.34142582, 0.84756607, + 0.53360225, 0.90053371, 0.78779561, 0.51604647, 0.35325637, + 0.51408926, 0.84489897, 0.55342816, 0.89405683, 0.54588131, + 0.96038024, 0.71349698, 0.43456735, 0.52462506, 0.43074793]) >>> f_regression(X, y) - (array([2.76445780e+00, 1.05267800e+01, 4.43399092e-02, 2.04580501e-02, - 3.13208557e-02, 1.35248025e-03]), array([0.11369388, 0.0044996 , 0.83558782, 0.88785417, 0.86150261, - 0.97106833])) + (array([ 6.86260598, 7.23175589, 24.11424725, 0.6605354 , 1.26266286, + 1.82421406]), array([1.73658700e-02, 1.49916659e-02, 1.12697153e-04, 4.26990301e-01, + 2.75911201e-01, 1.93549275e-01])) */ // scalastyle:on val data = Seq( - LabeledPoint(0.33997803, Vectors.dense(1.67318514e-01, 1.78398028e-01, 4.36846538e-01, - 5.24003164e-01, 1.80915415e-01, 1.98030859e-01)), - LabeledPoint(0.71456716, Vectors.dense(3.71836586e-01, 6.13453963e-01, 7.15269190e-01, - 9.33623792e-03, 5.36095674e-01, 2.74223333e-01)), - LabeledPoint(0.58676766, Vectors.dense(3.68988949e-01, 5.34104018e-01, 5.24858744e-01, - 6.86815853e-01, 3.26534757e-01, 6.92699400e-01)), - LabeledPoint(0.52894227, Vectors.dense(4.87748505e-02, 3.07080315e-01, 7.82955385e-01, - 6.90167375e-01, 6.44077919e-01, 4.23739024e-01)), - LabeledPoint(0.53158463, Vectors.dense(6.50153455e-01, 8.32746110e-01, 6.88029140e-03, - 1.27859556e-01, 6.80223767e-01, 6.25825675e-01)), - LabeledPoint(0.55515181, Vectors.dense(9.47343271e-01, 2.13193978e-01, 3.71342472e-01, - 8.21291956e-01, 4.38195693e-01, 5.76569439e-01)), - LabeledPoint(0.67008744, Vectors.dense(9.96499254e-01, 8.45833297e-01, 6.56086922e-02, - 5.90029174e-01, 1.68954572e-01, 7.19792823e-02)), - LabeledPoint(0.5966537, Vectors.dense(1.85926914e-01, 9.60329804e-01, 3.13487406e-01, - 9.59549928e-01, 6.89093311e-01, 6.94999427e-01)), - LabeledPoint(0.56255674, Vectors.dense(9.40164576e-01, 2.69042714e-02, 5.39491321e-01, - 5.74068666e-01, 1.10935343e-01, 2.17519760e-01)), - LabeledPoint(0.33904133, Vectors.dense(2.97951848e-02, 1.06592106e-01, 5.74931856e-01, - 8.80801522e-01, 8.60445070e-01, 9.22757966e-01)), - LabeledPoint(0.66485577, Vectors.dense(9.80970473e-01, 3.05909353e-01, 4.96401766e-01, - 2.44342697e-01, 6.90559227e-01, 5.64858704e-01)), - LabeledPoint(0.38514965, Vectors.dense(1.55939260e-01, 2.18626853e-01, 5.01834270e-01, - 1.86694987e-01, 9.15411148e-01, 6.40527848e-01)), - LabeledPoint(0.73885841, Vectors.dense(3.16107608e-01, 9.25906358e-01, 5.47327167e-01, - 4.83712979e-01, 8.42305220e-01, 7.58488462e-01)), - LabeledPoint(0.45766267, Vectors.dense(4.14393503e-01, 1.30817883e-01, 5.62034942e-01, - 1.05150633e-01, 5.35632795e-01, 9.47594074e-04)), - LabeledPoint(0.34801557, Vectors.dense(5.26233981e-01, 7.63781419e-02, 3.19188240e-01, - 5.16528633e-02, 5.28416724e-01, 6.47050470e-03)), - LabeledPoint(0.52529452, Vectors.dense(2.73404764e-01, 7.17070744e-01, 3.12889595e-01, - 8.39271965e-01, 9.67650889e-01, 8.50098873e-01)), - LabeledPoint(0.42503336, Vectors.dense(4.63289495e-01, 3.57055416e-02, 5.43528596e-01, - 4.44840919e-01, 9.36845855e-02, 7.81595037e-01)), - LabeledPoint(0.60221968, Vectors.dense(3.21784993e-01, 3.15622454e-01, 7.58870408e-01, - 5.18198558e-01, 2.28151905e-01, 4.42460325e-01)), - LabeledPoint(0.58964479, Vectors.dense(3.72428352e-01, 1.44447969e-01, 8.40274188e-01, - 5.86308041e-01, 6.09893953e-01, 3.97006473e-01)), - LabeledPoint(0.58194949, Vectors.dense(3.12776786e-01, 9.33630195e-01, 2.29328749e-01, - 4.32807208e-01, 1.51703470e-02, 1.51589320e-01))) + LabeledPoint(0.19775997, Vectors.dense(0.15266373, 0.30235661, 0.06203641, 0.45986034, + 0.83525338, 0.92699705)), + LabeledPoint(0.66009772, Vectors.dense(0.72698898, 0.76849622, 0.26920507, 0.64402929, + 0.09337326, 0.07968589)), + LabeledPoint(0.80865842, Vectors.dense(0.58961375, 0.34334054, 0.98887615, 0.62647321, + 0.68177928, 0.55225681)), + LabeledPoint(0.34142582, Vectors.dense(0.26886006, 0.37325939, 0.2229281, 0.1864426, + 0.39064809, 0.19316241)), + LabeledPoint(0.84756607, Vectors.dense(0.61091093, 0.88280845, 0.62233882, 0.25311894, + 0.17993031, 0.81640447)), + LabeledPoint(0.53360225, Vectors.dense(0.22537162, 0.51685714, 0.51849582, 0.60037494, + 0.53262048, 0.01331005)), + LabeledPoint(0.90053371, Vectors.dense(0.52409726, 0.89588471, 0.76990129, 0.1228517, + 0.29587269, 0.61202358)), + LabeledPoint(0.78779561, Vectors.dense(0.72613812, 0.46349747, 0.76911037, 0.19163103, + 0.55786672, 0.55077816)), + LabeledPoint(0.51604647, Vectors.dense(0.47222549, 0.79188496, 0.11524968, 0.6813039, + 0.36233361, 0.34420889)), + LabeledPoint(0.35325637, Vectors.dense(0.44951875, 0.02694226, 0.41524769, 0.9222317, + 0.09120557, 0.31512178)), + LabeledPoint(0.51408926, Vectors.dense(0.52802224, 0.32806203, 0.44891554, 0.01633442, + 0.0970269, 0.69258857)), + LabeledPoint(0.84489897, Vectors.dense(0.83594341, 0.42432199, 0.8487743, 0.54679121, + 0.35410346, 0.72724968)), + LabeledPoint(0.55342816, Vectors.dense(0.09385168, 0.8928588, 0.33625828, 0.89183268, + 0.296849, 0.30164829)), + LabeledPoint(0.89405683, Vectors.dense(0.80624061, 0.83760997, 0.63428133, 0.3113273, + 0.02944858, 0.39977732)), + LabeledPoint(0.54588131, Vectors.dense(0.51817346, 0.00738845, 0.77494778, 0.8544712, + 0.13153282, 0.28767364)), + LabeledPoint(0.96038024, Vectors.dense(0.32658881, 0.90655956, 0.99955954, 0.77088429, + 0.04284752, 0.96525111)), + LabeledPoint(0.71349698, Vectors.dense(0.97521246, 0.2025168, 0.67985305, 0.46534506, + 0.92001748, 0.72820735)), + LabeledPoint(0.43456735, Vectors.dense(0.24585653, 0.01953996, 0.70598881, 0.77448287, + 0.4729746, 0.80146736)), + LabeledPoint(0.52462506, Vectors.dense(0.17539792, 0.72016934, 0.3678759, 0.53209295, + 0.29719397, 0.37429151)), + LabeledPoint(0.43074793, Vectors.dense(0.72810013, 0.39850784, 0.1058295, 0.39858265, + 0.52196395, 0.1060125))) for (numParts <- List(2, 4, 6, 8)) { val df = spark.createDataFrame(sc.parallelize(data, numParts)) @@ -140,10 +141,10 @@ class FValueTestSuite val (pValues: Vector, fValues: Vector) = fRegression.select("pValues", "fValues") .as[(Vector, Vector)].head() - assert(pValues ~== Vectors.dense(0.11369388, 0.0044996, 0.83558782, 0.88785417, 0.86150261, - 0.97106833) relTol 1e-6) - assert(fValues ~== Vectors.dense(2.76445780e+00, 1.05267800e+01, 4.43399092e-02, - 2.04580501e-02, 3.13208557e-02, 1.35248025e-03) relTol 1e-6) + assert(pValues ~== Vectors.dense(1.73658700e-02, 1.49916659e-02, 1.12697153e-04, + 4.26990301e-01, 2.75911201e-01, 1.93549275e-01) relTol 1e-6) + assert(fValues ~== Vectors.dense(6.86260598, 7.23175589, 24.11424725, 0.6605354, 1.26266286, + 1.82421406) relTol 1e-6) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala index 6e68c1c9d36c8..12ab2ac3cc698 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala @@ -23,6 +23,12 @@ import org.apache.spark.mllib.util.TestingUtils._ class BLASSuite extends SparkFunSuite { + test("nativeL1Threshold") { + assert(getBLAS(128) == BLAS.f2jBLAS) + assert(getBLAS(256) == BLAS.nativeBLAS) + assert(getBLAS(512) == BLAS.nativeBLAS) + } + test("copy") { val sx = Vectors.sparse(4, Array(0, 2), Array(1.0, -2.0)) val dx = Vectors.dense(1.0, 0.0, -2.0, 0.0) diff --git a/pom.xml b/pom.xml index a3357596ff548..cd85db6e03264 100644 --- a/pom.xml +++ b/pom.xml @@ -178,7 +178,7 @@ 2.6.2 4.1.17 - 3.0.15 + 3.0.16 2.30 2.10.5 3.5.2 @@ -931,8 +931,9 @@ com.spotify docker-client - 5.0.2 + 8.14.1 test + shaded guava @@ -950,6 +951,12 @@ 5.1.38 test + + org.mariadb.jdbc + mariadb-java-client + 2.5.4 + test + org.postgresql postgresql diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bf051f60b7a57..f28ae5676a72c 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -55,8 +55,6 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ExecutorPlugin"), // [SPARK-28980][SQL][CORE][MLLIB] Remove more old deprecated items in Spark 3 - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.createExternalTable"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createExternalTable"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeans.train"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.clustering.KMeans.train"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.classification.LogisticRegressionWithSGD$"), @@ -244,14 +242,6 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.shuffleWriteTime"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.shuffleRecordsWritten"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.AccumulableInfo.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.approxCountDistinct"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.toRadians"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.toDegrees"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.monotonicallyIncreasingId"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.clearActive"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.getOrCreate"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.setActive"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.SQLContext.this"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.fMeasure"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.recall"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.precision"), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1a2a7c3066d26..44ef35b65a91f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -88,7 +88,8 @@ object SparkBuild extends PomBuild { val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty override val profiles = { - val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") match { + val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") + .orElse(Properties.propOrNone("sbt.maven.profiles")) match { case None => Seq("sbt") case Some(v) => v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq @@ -479,7 +480,8 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", - "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite" + "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite", + "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite" ) private val DEFAULT_TEST_GROUP = "default_test_group" @@ -623,7 +625,8 @@ object DependencyOverrides { dependencyOverrides += "com.google.guava" % "guava" % "14.0.1", dependencyOverrides += "commons-io" % "commons-io" % "2.4", dependencyOverrides += "xerces" % "xercesImpl" % "2.12.0", - dependencyOverrides += "jline" % "jline" % "2.14.6") + dependencyOverrides += "jline" % "jline" % "2.14.6", + dependencyOverrides += "org.apache.avro" % "avro" % "1.8.2") } /** diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 76a5bd0d645ba..70c0b27a6aa33 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -113,7 +113,7 @@ def wrapper(self, *args, **kwargs): # for back compatibility -from pyspark.sql import SQLContext, Row +from pyspark.sql import SQLContext, HiveContext, Row __all__ = [ "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", diff --git a/python/pyspark/find_spark_home.py b/python/pyspark/find_spark_home.py index 9c4ed46598632..52f6ea9a37100 100755 --- a/python/pyspark/find_spark_home.py +++ b/python/pyspark/find_spark_home.py @@ -40,6 +40,7 @@ def is_spark_home(path): paths = ["../", os.path.dirname(os.path.realpath(__file__))] # Add the path of the PySpark module if it exists + import_error_raised = False if sys.version < "3": import imp try: @@ -49,7 +50,7 @@ def is_spark_home(path): paths.append(os.path.join(module_home, "../../")) except ImportError: # Not pip installed no worries - pass + import_error_raised = True else: from importlib.util import find_spec try: @@ -59,7 +60,7 @@ def is_spark_home(path): paths.append(os.path.join(module_home, "../../")) except ImportError: # Not pip installed no worries - pass + import_error_raised = True # Normalize the paths paths = [os.path.abspath(p) for p in paths] @@ -68,6 +69,19 @@ def is_spark_home(path): return next(path for path in paths if is_spark_home(path)) except StopIteration: print("Could not find valid SPARK_HOME while searching {0}".format(paths), file=sys.stderr) + if import_error_raised: + print( + "\nDid you install PySpark via a package manager such as pip or Conda? If so,\n" + "PySpark was not found in your Python environment. It is possible your\n" + "Python environment does not properly bind with your package manager.\n" + "\nPlease check your default 'python' and if you set PYSPARK_PYTHON and/or\n" + "PYSPARK_DRIVER_PYTHON environment variables, and see if you can import\n" + "PySpark, for example, 'python -c 'import pyspark'.\n" + "\nIf you cannot import, you can install by using the Python executable directly,\n" + "for example, 'python -m pip install pyspark [--user]'. Otherwise, you can also\n" + "explicitly set the Python executable, that has PySpark installed, to\n" + "PYSPARK_PYTHON or PYSPARK_DRIVER_PYTHON environment variables, for example,\n" + "'PYSPARK_PYTHON=python3 pyspark'.\n", file=sys.stderr) sys.exit(-1) if __name__ == "__main__": diff --git a/python/pyspark/ml/base.py b/python/pyspark/ml/base.py index b8df5a3ca5c14..a24b1ca5122db 100644 --- a/python/pyspark/ml/base.py +++ b/python/pyspark/ml/base.py @@ -96,8 +96,6 @@ def fitMultiple(self, dataset, paramMaps): :return: A thread safe iterable which contains one model for each param map. Each call to `next(modelIterator)` will return `(index, model)` where model was fit using `paramMaps[index]`. `index` values may not be sequential. - - .. note:: DeveloperApi """ estimator = self.copy() diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index fe61f9f0fffd6..1be8755c7b982 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -76,8 +76,6 @@ def __eq__(self, other): class TypeConverters(object): """ - .. note:: DeveloperApi - Factory methods for common type conversion functions for `Param.typeConverter`. .. versionadded:: 2.0.0 diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 097530230cbca..09e0748ffbb3b 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -324,8 +324,6 @@ def _to_java(self): @inherit_doc class PipelineSharedReadWrite(): """ - .. note:: DeveloperApi - Functions for :py:class:`MLReader` and :py:class:`MLWriter` shared between :py:class:`Pipeline` and :py:class:`PipelineModel` diff --git a/python/pyspark/ml/stat.py b/python/pyspark/ml/stat.py index 53a57af4a5be9..058146935ed91 100644 --- a/python/pyspark/ml/stat.py +++ b/python/pyspark/ml/stat.py @@ -411,6 +411,106 @@ def __init__(self, mean, cov): self.cov = cov +class ANOVATest(object): + """ + Conduct ANOVA Classification Test for continuous features against categorical labels. + + .. versionadded:: 3.1.0 + """ + @staticmethod + @since("3.1.0") + def test(dataset, featuresCol, labelCol): + """ + Perform an ANOVA test using dataset. + + :param dataset: + DataFrame of categorical labels and continuous features. + :param featuresCol: + Name of features column in dataset, of type `Vector` (`VectorUDT`). + :param labelCol: + Name of label column in dataset, of any numerical type. + :return: + DataFrame containing the test result for every feature against the label. + This DataFrame will contain a single Row with the following fields: + - `pValues: Vector` + - `degreesOfFreedom: Array[Long]` + - `fValues: Vector` + Each of these fields has one value per feature. + + >>> from pyspark.ml.linalg import Vectors + >>> from pyspark.ml.stat import ANOVATest + >>> dataset = [[2.0, Vectors.dense([0.43486404, 0.57153633, 0.43175686, + ... 0.51418671, 0.61632374, 0.96565515])], + ... [1.0, Vectors.dense([0.49162732, 0.6785187, 0.85460572, + ... 0.59784822, 0.12394819, 0.53783355])], + ... [2.0, Vectors.dense([0.30879653, 0.54904515, 0.17103889, + ... 0.40492506, 0.18957493, 0.5440016])], + ... [3.0, Vectors.dense([0.68114391, 0.60549825, 0.69094651, + ... 0.62102109, 0.05471483, 0.96449167])]] + >>> dataset = spark.createDataFrame(dataset, ["label", "features"]) + >>> anovaResult = ANOVATest.test(dataset, 'features', 'label') + >>> row = anovaResult.select("fValues", "pValues").collect() + >>> row[0].fValues + DenseVector([4.0264, 18.4713, 3.4659, 1.9042, 0.5532, 0.512]) + >>> row[0].pValues + DenseVector([0.3324, 0.1623, 0.3551, 0.456, 0.689, 0.7029]) + """ + sc = SparkContext._active_spark_context + javaTestObj = _jvm().org.apache.spark.ml.stat.ANOVATest + args = [_py2java(sc, arg) for arg in (dataset, featuresCol, labelCol)] + return _java2py(sc, javaTestObj.test(*args)) + + +class FValueTest(object): + """ + Conduct F Regression test for continuous features against continuous labels. + + .. versionadded:: 3.1.0 + """ + @staticmethod + @since("3.1.0") + def test(dataset, featuresCol, labelCol): + """ + Perform a F Regression test using dataset. + + :param dataset: + DataFrame of continuous labels and continuous features. + :param featuresCol: + Name of features column in dataset, of type `Vector` (`VectorUDT`). + :param labelCol: + Name of label column in dataset, of any numerical type. + :return: + DataFrame containing the test result for every feature against the label. + This DataFrame will contain a single Row with the following fields: + - `pValues: Vector` + - `degreesOfFreedom: Array[Long]` + - `fValues: Vector` + Each of these fields has one value per feature. + + >>> from pyspark.ml.linalg import Vectors + >>> from pyspark.ml.stat import FValueTest + >>> dataset = [[0.57495218, Vectors.dense([0.43486404, 0.57153633, 0.43175686, + ... 0.51418671, 0.61632374, 0.96565515])], + ... [0.84619853, Vectors.dense([0.49162732, 0.6785187, 0.85460572, + ... 0.59784822, 0.12394819, 0.53783355])], + ... [0.39777647, Vectors.dense([0.30879653, 0.54904515, 0.17103889, + ... 0.40492506, 0.18957493, 0.5440016])], + ... [0.79201573, Vectors.dense([0.68114391, 0.60549825, 0.69094651, + ... 0.62102109, 0.05471483, 0.96449167])]] + >>> dataset = spark.createDataFrame(dataset, ["label", "features"]) + >>> fValueResult = FValueTest.test(dataset, 'features', 'label') + >>> row = fValueResult.select("fValues", "pValues").collect() + >>> row[0].fValues + DenseVector([3.741, 7.5807, 142.0684, 34.9849, 0.4112, 0.0539]) + >>> row[0].pValues + DenseVector([0.1928, 0.1105, 0.007, 0.0274, 0.5871, 0.838]) + """ + sc = SparkContext._active_spark_context + javaTestObj = _jvm().org.apache.spark.ml.stat.FValueTest + args = [_py2java(sc, arg) for arg in (dataset, featuresCol, labelCol)] + return _java2py(sc, javaTestObj.test(*args)) + + if __name__ == "__main__": import doctest import numpy diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index fe0ed569f214d..35ad5518e1c1f 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -345,8 +345,6 @@ def read(cls): @inherit_doc class DefaultParamsWritable(MLWritable): """ - .. note:: DeveloperApi - Helper trait for making simple :py:class:`Params` types writable. If a :py:class:`Params` class stores all data as :py:class:`Param` values, then extending this trait will provide a default implementation of writing saved instances of the class. @@ -370,8 +368,6 @@ def write(self): @inherit_doc class DefaultParamsWriter(MLWriter): """ - .. note:: DeveloperApi - Specialization of :py:class:`MLWriter` for :py:class:`Params` types Class for writing Estimators and Transformers whose parameters are JSON-serializable. @@ -445,8 +441,6 @@ def _get_metadata_to_save(instance, sc, extraMetadata=None, paramMap=None): @inherit_doc class DefaultParamsReadable(MLReadable): """ - .. note:: DeveloperApi - Helper trait for making simple :py:class:`Params` types readable. If a :py:class:`Params` class stores all data as :py:class:`Param` values, then extending this trait will provide a default implementation of reading saved @@ -466,8 +460,6 @@ def read(cls): @inherit_doc class DefaultParamsReader(MLReader): """ - .. note:: DeveloperApi - Specialization of :py:class:`MLReader` for :py:class:`Params` types Default :py:class:`MLReader` implementation for transformers and estimators that diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 905c4da2d880d..3efae6ff0ecc3 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -43,8 +43,6 @@ class VectorTransformer(object): """ - .. note:: DeveloperApi - Base class for transformation of a vector or RDD of vector """ def transform(self, vector): diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index 0a8d71c12e060..c28cb8c3b9cbe 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -43,7 +43,7 @@ from pyspark.sql.types import Row -from pyspark.sql.context import SQLContext, UDFRegistration +from pyspark.sql.context import SQLContext, HiveContext, UDFRegistration from pyspark.sql.session import SparkSession from pyspark.sql.column import Column from pyspark.sql.catalog import Catalog @@ -55,7 +55,7 @@ __all__ = [ - 'SparkSession', 'SQLContext', 'UDFRegistration', + 'SparkSession', 'SQLContext', 'HiveContext', 'UDFRegistration', 'DataFrame', 'GroupedData', 'Column', 'Catalog', 'Row', 'DataFrameNaFunctions', 'DataFrameStatFunctions', 'Window', 'WindowSpec', 'DataFrameReader', 'DataFrameWriter', 'PandasCogroupedOps' diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 08cf6ee330785..974251f63b37a 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -138,6 +138,26 @@ def listColumns(self, tableName, dbName=None): isBucket=jcolumn.isBucket())) return columns + @since(2.0) + def createExternalTable(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. + + 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. diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index e9f6a62feb6d2..956343a2310b8 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -32,7 +32,7 @@ from pyspark.sql.udf import UDFRegistration from pyspark.sql.utils import install_exception_handler -__all__ = ["SQLContext"] +__all__ = ["SQLContext", "HiveContext"] class SQLContext(object): @@ -340,6 +340,24 @@ def dropTempTable(self, tableName): """ self.sparkSession.catalog.dropTempView(tableName) + @since(1.3) + def createExternalTable(self, tableName, path=None, source=None, schema=None, **options): + """Creates an external 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` + """ + return self.sparkSession.catalog.createExternalTable( + tableName, path, source, schema, **options) + @ignore_unicode_prefix @since(1.0) def sql(self, sqlQuery): @@ -463,6 +481,53 @@ def streams(self): return StreamingQueryManager(self._ssql_ctx.streams()) +class HiveContext(SQLContext): + """A variant of Spark SQL that integrates with data stored in Hive. + + Configuration for Hive is read from ``hive-site.xml`` on the classpath. + It supports running both SQL and HiveQL commands. + + :param sparkContext: The SparkContext to wrap. + :param jhiveContext: An optional JVM Scala HiveContext. If set, we do not instantiate a new + :class:`HiveContext` in the JVM, instead we make all calls to this object. + + .. note:: Deprecated in 2.0.0. Use SparkSession.builder.enableHiveSupport().getOrCreate(). + """ + + def __init__(self, sparkContext, jhiveContext=None): + warnings.warn( + "HiveContext is deprecated in Spark 2.0.0. Please use " + + "SparkSession.builder.enableHiveSupport().getOrCreate() instead.", + DeprecationWarning) + if jhiveContext is None: + sparkContext._conf.set("spark.sql.catalogImplementation", "hive") + sparkSession = SparkSession.builder._sparkContext(sparkContext).getOrCreate() + else: + sparkSession = SparkSession(sparkContext, jhiveContext.sparkSession()) + SQLContext.__init__(self, sparkContext, sparkSession, jhiveContext) + + @classmethod + def _createForTesting(cls, sparkContext): + """(Internal use only) Create a new HiveContext for testing. + + All test code that touches HiveContext *must* go through this method. Otherwise, + you may end up launching multiple derby instances and encounter with incredibly + confusing error messages. + """ + jsc = sparkContext._jsc.sc() + jtestHive = sparkContext._jvm.org.apache.spark.sql.hive.test.TestHiveContext(jsc, False) + return cls(sparkContext, jtestHive) + + def refreshTable(self, tableName): + """Invalidate and refresh all the cached the metadata of the given + table. For performance reasons, Spark SQL or the external data source + library it uses might cache certain metadata about a table, such as the + location of blocks. When those change outside of Spark SQL, users should + call this function to invalidate the cache. + """ + self._ssql_ctx.refreshTable(tableName) + + def _test(): import os import doctest diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 971cdb128ee90..78b574685327c 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -122,6 +122,25 @@ def toJSON(self, use_unicode=True): rdd = self._jdf.toJSON() return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode)) + @since(1.3) + def registerTempTable(self, name): + """Registers this DataFrame as a temporary table using the given name. + + The lifetime of this temporary table is tied to the :class:`SparkSession` + that was used to create this :class:`DataFrame`. + + >>> df.registerTempTable("people") + >>> df2 = spark.sql("select * from people") + >>> sorted(df.collect()) == sorted(df2.collect()) + True + >>> spark.catalog.dropTempView("people") + + .. note:: Deprecated in 2.0, use createOrReplaceTempView instead. + """ + warnings.warn( + "Deprecated in 2.0, use createOrReplaceTempView instead.", DeprecationWarning) + self._jdf.createOrReplaceTempView(name) + @since(2.0) def createTempView(self, name): """Creates a local temporary view with this :class:`DataFrame`. diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index cd6a832ff5147..de0d38e2aed12 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -296,6 +296,8 @@ def _options_to_str(options): # Wraps deprecated functions (keys) with the messages (values). _functions_deprecated = { + 'toDegrees': 'Deprecated in 2.1, use degrees instead.', + 'toRadians': 'Deprecated in 2.1, use radians instead.', } for _name, _doc in _functions.items(): @@ -319,6 +321,15 @@ def _options_to_str(options): del _name, _doc +@since(1.3) +def approxCountDistinct(col, rsd=None): + """ + .. note:: Deprecated in 2.1, use :func:`approx_count_distinct` instead. + """ + warnings.warn("Deprecated in 2.1, use approx_count_distinct instead.", DeprecationWarning) + return approx_count_distinct(col, rsd) + + @since(2.1) def approx_count_distinct(col, rsd=None): """Aggregate function: returns a new :class:`Column` for approximate distinct count of @@ -584,11 +595,64 @@ def nanvl(col1, col2): return Column(sc._jvm.functions.nanvl(_to_java_column(col1), _to_java_column(col2))) +@since(3.1) +def percentile_approx(col, percentage, accuracy=10000): + """Returns the approximate percentile value of numeric column col at the given percentage. + The value of percentage must be between 0.0 and 1.0. + + The accuracy parameter (default: 10000) + is a positive numeric literal which controls approximation accuracy at the cost of memory. + Higher value of accuracy yields better accuracy, 1.0/accuracy is the relative error + of the approximation. + + When percentage is an array, each value of the percentage array must be between 0.0 and 1.0. + In this case, returns the approximate percentile array of column col + at the given percentage array. + + >>> key = (col("id") % 3).alias("key") + >>> value = (randn(42) + key * 10).alias("value") + >>> df = spark.range(0, 1000, 1, 1).select(key, value) + >>> df.select( + ... percentile_approx("value", [0.25, 0.5, 0.75], 1000000).alias("quantiles") + ... ).printSchema() + root + |-- quantiles: array (nullable = true) + | |-- element: double (containsNull = false) + + >>> df.groupBy("key").agg( + ... percentile_approx("value", 0.5, lit(1000000)).alias("median") + ... ).printSchema() + root + |-- key: long (nullable = true) + |-- median: double (nullable = true) + """ + sc = SparkContext._active_spark_context + + if isinstance(percentage, (list, tuple)): + # A local list + percentage = sc._jvm.functions.array(_to_seq(sc, [ + _create_column_from_literal(x) for x in percentage + ])) + elif isinstance(percentage, Column): + # Already a Column + percentage = _to_java_column(percentage) + else: + # Probably scalar + percentage = _create_column_from_literal(percentage) + + accuracy = ( + _to_java_column(accuracy) if isinstance(accuracy, Column) + else _create_column_from_literal(accuracy) + ) + + return Column(sc._jvm.functions.percentile_approx(_to_java_column(col), percentage, accuracy)) + + @ignore_unicode_prefix @since(1.4) def rand(seed=None): """Generates a random column with independent and identically distributed (i.i.d.) samples - from U[0.0, 1.0]. + uniformly distributed in [0.0, 1.0). .. note:: The function is non-deterministic in general case. @@ -920,8 +984,9 @@ def date_format(date, format): format given by the second argument. A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All - pattern letters of the Java class `java.time.format.DateTimeFormatter` can be used. + pattern letters of `datetime pattern`_. can be used. + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html .. note:: Use when ever possible specialized functions like `year`. These benefit from a specialized implementation. @@ -1138,8 +1203,7 @@ def months_between(date1, date2, roundOff=True): @since(2.2) def to_date(col, format=None): """Converts a :class:`Column` into :class:`pyspark.sql.types.DateType` - using the optionally specified format. Specify formats according to - `DateTimeFormatter `_. # noqa + using the optionally specified format. Specify formats according to `datetime pattern`_. By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format is omitted. Equivalent to ``col.cast("date")``. @@ -1162,8 +1226,7 @@ def to_date(col, format=None): @since(2.2) def to_timestamp(col, format=None): """Converts a :class:`Column` into :class:`pyspark.sql.types.TimestampType` - using the optionally specified format. Specify formats according to - `DateTimeFormatter `_. # noqa + using the optionally specified format. Specify formats according to `datetime pattern`_. By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format is omitted. Equivalent to ``col.cast("timestamp")``. @@ -1303,7 +1366,12 @@ def from_utc_timestamp(timestamp, tz): timestamp to string according to the session local timezone. :param timestamp: the column that contains timestamps - :param tz: a string that has the ID of timezone, e.g. "GMT", "America/Los_Angeles", etc + :param tz: A string detailing the time zone ID that the input should be adjusted to. It should + be in the format of either region-based zone IDs or zone offsets. Region IDs must + have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in + the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are + supported as aliases of '+00:00'. Other short names are not recommended to use + because they can be ambiguous. .. versionchanged:: 2.4 `tz` can take a :class:`Column` containing timezone ID strings. @@ -1337,7 +1405,12 @@ def to_utc_timestamp(timestamp, tz): timestamp to string according to the session local timezone. :param timestamp: the column that contains timestamps - :param tz: a string that has the ID of timezone, e.g. "GMT", "America/Los_Angeles", etc + :param tz: A string detailing the time zone ID that the input should be adjusted to. It should + be in the format of either region-based zone IDs or zone offsets. Region IDs must + have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in + the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are + supported as aliases of '+00:00'. Other short names are not recommended to use + because they can be ambiguous. .. versionchanged:: 2.4 `tz` can take a :class:`Column` containing timezone ID strings. diff --git a/python/pyspark/sql/pandas/conversion.py b/python/pyspark/sql/pandas/conversion.py index 8548cd222bf10..251625ae412cf 100644 --- a/python/pyspark/sql/pandas/conversion.py +++ b/python/pyspark/sql/pandas/conversion.py @@ -21,6 +21,7 @@ xrange = range else: from itertools import izip as zip +from collections import Counter from pyspark import since from pyspark.rdd import _load_from_socket @@ -102,13 +103,17 @@ def toPandas(self): try: from pyspark.sql.pandas.types import _check_series_localize_timestamps import pyarrow - batches = self._collect_as_arrow() + # Rename columns to avoid duplicated column names. + tmp_column_names = ['col_{}'.format(i) for i in range(len(self.columns))] + batches = self.toDF(*tmp_column_names)._collect_as_arrow() if len(batches) > 0: table = pyarrow.Table.from_batches(batches) # Pandas DataFrame created from PyArrow uses datetime64[ns] for date type # values, but we should use datetime.date to match the behavior with when # Arrow optimization is disabled. pdf = table.to_pandas(date_as_object=True) + # Rename back to the original column names. + pdf.columns = self.columns for field in self.schema: if isinstance(field.dataType, TimestampType): pdf[field.name] = \ @@ -131,9 +136,16 @@ def toPandas(self): # Below is toPandas without Arrow optimization. pdf = pd.DataFrame.from_records(self.collect(), columns=self.columns) + column_counter = Counter(self.columns) + + dtype = [None] * len(self.schema) + for fieldIdx, field in enumerate(self.schema): + # For duplicate column name, we use `iloc` to access it. + if column_counter[field.name] > 1: + pandas_col = pdf.iloc[:, fieldIdx] + else: + pandas_col = pdf[field.name] - dtype = {} - for field in self.schema: pandas_type = PandasConversionMixin._to_corrected_pandas_type(field.dataType) # SPARK-21766: if an integer field is nullable and has null values, it can be # inferred by pandas as float column. Once we convert the column with NaN back @@ -141,16 +153,36 @@ def toPandas(self): # float type, not the corrected type from the schema in this case. if pandas_type is not None and \ not(isinstance(field.dataType, IntegralType) and field.nullable and - pdf[field.name].isnull().any()): - dtype[field.name] = pandas_type + pandas_col.isnull().any()): + dtype[fieldIdx] = pandas_type # Ensure we fall back to nullable numpy types, even when whole column is null: - if isinstance(field.dataType, IntegralType) and pdf[field.name].isnull().any(): - dtype[field.name] = np.float64 - if isinstance(field.dataType, BooleanType) and pdf[field.name].isnull().any(): - dtype[field.name] = np.object + if isinstance(field.dataType, IntegralType) and pandas_col.isnull().any(): + dtype[fieldIdx] = np.float64 + if isinstance(field.dataType, BooleanType) and pandas_col.isnull().any(): + dtype[fieldIdx] = np.object + + df = pd.DataFrame() + for index, t in enumerate(dtype): + column_name = self.schema[index].name + + # For duplicate column name, we use `iloc` to access it. + if column_counter[column_name] > 1: + series = pdf.iloc[:, index] + else: + series = pdf[column_name] + + if t is not None: + series = series.astype(t, copy=False) + + # `insert` API makes copy of data, we only do it for Series of duplicate column names. + # `pdf.iloc[:, index] = pdf.iloc[:, index]...` doesn't always work because `iloc` could + # return a view or a copy depending by context. + if column_counter[column_name] > 1: + df.insert(index, column_name, series, allow_duplicates=True) + else: + df[column_name] = series - for f, t in dtype.items(): - pdf[f] = pdf[f].astype(t, copy=False) + pdf = df if timezone is None: return pdf diff --git a/python/pyspark/sql/pandas/functions.py b/python/pyspark/sql/pandas/functions.py index 31aa321bf5826..094dc357b6822 100644 --- a/python/pyspark/sql/pandas/functions.py +++ b/python/pyspark/sql/pandas/functions.py @@ -150,10 +150,11 @@ def pandas_udf(f=None, returnType=None, functionType=None): The function takes an iterator of `pandas.Series` and outputs an iterator of `pandas.Series`. In this case, the created pandas UDF instance requires one input - column when this is called as a PySpark column. The output of each series from - the function should always be of the same length as the input. + column when this is called as a PySpark column. The length of the entire output from + the function should be the same length of the entire input; therefore, it can + prefetch the data from the input iterator as long as the lengths are the same. - It is useful when the UDF execution + It is also useful when the UDF execution requires initializing some states although internally it works identically as Series to Series case. The pseudocode below illustrates the example. @@ -194,9 +195,8 @@ def calculate(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]: The function takes an iterator of a tuple of multiple `pandas.Series` and outputs an iterator of `pandas.Series`. In this case, the created pandas UDF instance requires input columns as many as the series when this is called as a PySpark column. - It works identically as Iterator of Series to Iterator of Series case except - the parameter difference. The output of each series from the function should always - be of the same length as the input. + Otherwise, it has the same characteristics and restrictions as Iterator of Series + to Iterator of Series case. >>> from typing import Iterator, Tuple >>> from pyspark.sql.functions import struct, col @@ -384,6 +384,14 @@ def _create_pandas_udf(f, returnType, evalType): "In Python 3.6+ and Spark 3.0+, it is preferred to specify type hints for " "pandas UDF instead of specifying pandas UDF type which will be deprecated " "in the future releases. See SPARK-28264 for more details.", UserWarning) + elif evalType in [PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, + PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF]: + # In case of 'SQL_GROUPED_MAP_PANDAS_UDF', deprecation warning is being triggered + # at `apply` instead. + # In case of 'SQL_MAP_PANDAS_ITER_UDF' and 'SQL_COGROUPED_MAP_PANDAS_UDF', the + # evaluation type will always be set. + pass elif len(argspec.annotations) > 0: evalType = infer_eval_type(signature(f)) assert evalType is not None diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 5904688e087a7..6ad6377288ec5 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -105,9 +105,18 @@ def option(self, key, value): """Adds an input option for the underlying data source. You can set the following option(s) for reading files: - * ``timeZone``: sets the string that indicates a timezone to be used to parse timestamps - in the JSON/CSV datasources or partition values. - If it isn't set, it uses the default value, session local timezone. + * ``timeZone``: sets the string that indicates a time zone ID to be used to parse + timestamps in the JSON/CSV datasources or partition values. The following + formats of `timeZone` are supported: + + * Region-based zone ID: It should have the form 'area/city', such as \ + 'America/Los_Angeles'. + * Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or \ + '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. + + Other short names like 'CST' are not recommended to use because they can be + ambiguous. If it isn't set, the current value of the SQL config + ``spark.sql.session.timeZone`` is used by default. * ``pathGlobFilter``: an optional glob pattern to only include files with paths matching the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. It does not change the behavior of partition discovery. @@ -120,9 +129,18 @@ def options(self, **options): """Adds input options for the underlying data source. You can set the following option(s) for reading files: - * ``timeZone``: sets the string that indicates a timezone to be used to parse timestamps - in the JSON/CSV datasources or partition values. - If it isn't set, it uses the default value, session local timezone. + * ``timeZone``: sets the string that indicates a time zone ID to be used to parse + timestamps in the JSON/CSV datasources or partition values. The following + formats of `timeZone` are supported: + + * Region-based zone ID: It should have the form 'area/city', such as \ + 'America/Los_Angeles'. + * Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or \ + '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. + + Other short names like 'CST' are not recommended to use because they can be + ambiguous. If it isn't set, the current value of the SQL config + ``spark.sql.session.timeZone`` is used by default. * ``pathGlobFilter``: an optional glob pattern to only include files with paths matching the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. It does not change the behavior of partition discovery. @@ -221,14 +239,13 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, it uses the value specified in ``spark.sql.columnNameOfCorruptRecord``. :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.time.format.DateTimeFormatter``. This - applies to date type. If None is set, it uses the + follow the formats at `datetime pattern`_. + This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. :param timestampFormat: sets the string that indicates a timestamp format. - Custom date formats follow the formats at - ``java.time.format.DateTimeFormatter``. + Custom date formats follow the formats at `datetime pattern`_. This applies to timestamp type. If None is set, it uses the - default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. + default value, ``yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]``. :param multiLine: parse one record, which may span multiple lines, per file. If None is set, it uses the default value, ``false``. :param allowUnquotedControlChars: allows JSON Strings to contain unquoted control @@ -254,7 +271,9 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. - .. _partition discovery: /sql-data-sources-parquet.html#partition-discovery + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html >>> df1 = spark.read.json('python/test_support/sql/people.json') >>> df1.dtypes @@ -430,14 +449,13 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param negativeInf: sets the string representation of a negative infinity value. If None is set, it uses the default value, ``Inf``. :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.time.format.DateTimeFormatter``. This - applies to date type. If None is set, it uses the + follow the formats at `datetime pattern`_. + This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. :param timestampFormat: sets the string that indicates a timestamp format. - Custom date formats follow the formats at - ``java.time.format.DateTimeFormatter``. + Custom date formats follow the formats at `datetime pattern`_. This applies to timestamp type. If None is set, it uses the - default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. + default value, ``yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]``. :param maxColumns: defines a hard limit of how many columns a record can have. If None is set, it uses the default value, ``20480``. :param maxCharsPerColumn: defines the maximum number of characters allowed for any given @@ -665,9 +683,18 @@ def option(self, key, value): """Adds an output option for the underlying data source. You can set the following option(s) for writing files: - * ``timeZone``: sets the string that indicates a timezone to be used to format - timestamps in the JSON/CSV datasources or partition values. - If it isn't set, it uses the default value, session local timezone. + * ``timeZone``: sets the string that indicates a time zone ID to be used to format + timestamps in the JSON/CSV datasources or partition values. The following + formats of `timeZone` are supported: + + * Region-based zone ID: It should have the form 'area/city', such as \ + 'America/Los_Angeles'. + * Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or \ + '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. + + Other short names like 'CST' are not recommended to use because they can be + ambiguous. If it isn't set, the current value of the SQL config + ``spark.sql.session.timeZone`` is used by default. """ self._jwrite = self._jwrite.option(key, to_str(value)) return self @@ -677,9 +704,18 @@ def options(self, **options): """Adds output options for the underlying data source. You can set the following option(s) for writing files: - * ``timeZone``: sets the string that indicates a timezone to be used to format - timestamps in the JSON/CSV datasources or partition values. - If it isn't set, it uses the default value, session local timezone. + * ``timeZone``: sets the string that indicates a time zone ID to be used to format + timestamps in the JSON/CSV datasources or partition values. The following + formats of `timeZone` are supported: + + * Region-based zone ID: It should have the form 'area/city', such as \ + 'America/Los_Angeles'. + * Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or \ + '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. + + Other short names like 'CST' are not recommended to use because they can be + ambiguous. If it isn't set, the current value of the SQL config + ``spark.sql.session.timeZone`` is used by default. """ for k in options: self._jwrite = self._jwrite.option(k, to_str(options[k])) @@ -850,14 +886,13 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.time.format.DateTimeFormatter``. This - applies to date type. If None is set, it uses the + follow the formats at `datetime pattern`_. + This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. :param timestampFormat: sets the string that indicates a timestamp format. - Custom date formats follow the formats at - ``java.time.format.DateTimeFormatter``. + Custom date formats follow the formats at `datetime pattern`_. This applies to timestamp type. If None is set, it uses the - default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. + default value, ``yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]``. :param encoding: specifies encoding (charset) of saved json files. If None is set, the default UTF-8 charset will be used. :param lineSep: defines the line separator that should be used for writing. If None is @@ -954,15 +989,14 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No the default value, ``false``. :param nullValue: sets the string representation of a null value. If None is set, it uses the default value, empty string. - :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.time.format.DateTimeFormatter``. This - applies to date type. If None is set, it uses the + :param dateFormat: sets the string that indicates a date format. Custom date formats follow + the formats at `datetime pattern`_. + This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. :param timestampFormat: sets the string that indicates a timestamp format. - Custom date formats follow the formats at - ``java.time.format.DateTimeFormatter``. + Custom date formats follow the formats at `datetime pattern`_. This applies to timestamp type. If None is set, it uses the - default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. + default value, ``yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]``. :param ignoreLeadingWhiteSpace: a flag indicating whether or not leading whitespaces from values being written should be skipped. If None is set, it uses the default value, ``true``. diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 6a7624f2fbf41..05cf331d897a2 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -339,9 +339,18 @@ def option(self, key, value): """Adds an input option for the underlying data source. You can set the following option(s) for reading files: - * ``timeZone``: sets the string that indicates a timezone to be used to parse timestamps - in the JSON/CSV datasources or partition values. - If it isn't set, it uses the default value, session local timezone. + * ``timeZone``: sets the string that indicates a time zone ID to be used to parse + timestamps in the JSON/CSV datasources or partition values. The following + formats of `timeZone` are supported: + + * Region-based zone ID: It should have the form 'area/city', such as \ + 'America/Los_Angeles'. + * Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or \ + '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. + + Other short names like 'CST' are not recommended to use because they can be + ambiguous. If it isn't set, the current value of the SQL config + ``spark.sql.session.timeZone`` is used by default. .. note:: Evolving. @@ -355,9 +364,18 @@ def options(self, **options): """Adds input options for the underlying data source. You can set the following option(s) for reading files: - * ``timeZone``: sets the string that indicates a timezone to be used to parse timestamps - in the JSON/CSV datasources or partition values. - If it isn't set, it uses the default value, session local timezone. + * ``timeZone``: sets the string that indicates a time zone ID to be used to parse + timestamps in the JSON/CSV datasources or partition values. The following + formats of `timeZone` are supported: + + * Region-based zone ID: It should have the form 'area/city', such as \ + 'America/Los_Angeles'. + * Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or \ + '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. + + Other short names like 'CST' are not recommended to use because they can be + ambiguous. If it isn't set, the current value of the SQL config + ``spark.sql.session.timeZone`` is used by default. .. note:: Evolving. @@ -459,14 +477,13 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, it uses the value specified in ``spark.sql.columnNameOfCorruptRecord``. :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.time.format.DateTimeFormatter``. This - applies to date type. If None is set, it uses the + follow the formats at `datetime pattern`_. + This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. :param timestampFormat: sets the string that indicates a timestamp format. - Custom date formats follow the formats at - ``java.time.format.DateTimeFormatter``. + Custom date formats follow the formats at `datetime pattern`_. This applies to timestamp type. If None is set, it uses the - default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. + default value, ``yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]``. :param multiLine: parse one record, which may span multiple lines, per file. If None is set, it uses the default value, ``false``. :param allowUnquotedControlChars: allows JSON Strings to contain unquoted control @@ -490,7 +507,9 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. - .. _partition discovery: /sql-data-sources-parquet.html#partition-discovery + .. _partition discovery: + https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#partition-discovery + .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html >>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema) >>> json_sdf.isStreaming @@ -671,14 +690,13 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param negativeInf: sets the string representation of a negative infinity value. If None is set, it uses the default value, ``Inf``. :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.time.format.DateTimeFormatter``. This - applies to date type. If None is set, it uses the + follow the formats at `datetime pattern`_. + This applies to date type. If None is set, it uses the default value, ``yyyy-MM-dd``. :param timestampFormat: sets the string that indicates a timestamp format. - Custom date formats follow the formats at - ``java.time.format.DateTimeFormatter``. + Custom date formats follow the formats at `datetime pattern`_. This applies to timestamp type. If None is set, it uses the - default value, ``yyyy-MM-dd'T'HH:mm:ss.SSSXXX``. + default value, ``yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]``. :param maxColumns: defines a hard limit of how many columns a record can have. If None is set, it uses the default value, ``20480``. :param maxCharsPerColumn: defines the maximum number of characters allowed for any given @@ -812,9 +830,18 @@ def option(self, key, value): """Adds an output option for the underlying data source. You can set the following option(s) for writing files: - * ``timeZone``: sets the string that indicates a timezone to be used to format - timestamps in the JSON/CSV datasources or partition values. - If it isn't set, it uses the default value, session local timezone. + * ``timeZone``: sets the string that indicates a time zone ID to be used to format + timestamps in the JSON/CSV datasources or partition values. The following + formats of `timeZone` are supported: + + * Region-based zone ID: It should have the form 'area/city', such as \ + 'America/Los_Angeles'. + * Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or \ + '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. + + Other short names like 'CST' are not recommended to use because they can be + ambiguous. If it isn't set, the current value of the SQL config + ``spark.sql.session.timeZone`` is used by default. .. note:: Evolving. """ @@ -826,9 +853,18 @@ def options(self, **options): """Adds output options for the underlying data source. You can set the following option(s) for writing files: - * ``timeZone``: sets the string that indicates a timezone to be used to format - timestamps in the JSON/CSV datasources or partition values. - If it isn't set, it uses the default value, session local timezone. + * ``timeZone``: sets the string that indicates a time zone ID to be used to format + timestamps in the JSON/CSV datasources or partition values. The following + formats of `timeZone` are supported: + + * Region-based zone ID: It should have the form 'area/city', such as \ + 'America/Los_Angeles'. + * Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or \ + '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. + + Other short names like 'CST' are not recommended to use because they can be + ambiguous. If it isn't set, the current value of the SQL config + ``spark.sql.session.timeZone`` is used by default. .. note:: Evolving. """ diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 942cd4b4b0ea3..9861178158f85 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -529,6 +529,39 @@ def test_to_pandas(self): self.assertEquals(types[4], np.object) # datetime.date self.assertEquals(types[5], 'datetime64[ns]') + @unittest.skipIf(not have_pandas, pandas_requirement_message) + def test_to_pandas_with_duplicated_column_names(self): + import numpy as np + + sql = "select 1 v, 1 v" + for arrowEnabled in [False, True]: + with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": arrowEnabled}): + df = self.spark.sql(sql) + pdf = df.toPandas() + types = pdf.dtypes + self.assertEquals(types.iloc[0], np.int32) + self.assertEquals(types.iloc[1], np.int32) + + @unittest.skipIf(not have_pandas, pandas_requirement_message) + def test_to_pandas_on_cross_join(self): + import numpy as np + + sql = """ + select t1.*, t2.* from ( + select explode(sequence(1, 3)) v + ) t1 left join ( + select explode(sequence(1, 3)) v + ) t2 + """ + for arrowEnabled in [False, True]: + with self.sql_conf({"spark.sql.crossJoin.enabled": True, + "spark.sql.execution.arrow.pyspark.enabled": arrowEnabled}): + df = self.spark.sql(sql) + pdf = df.toPandas() + types = pdf.dtypes + self.assertEquals(types.iloc[0], np.int32) + self.assertEquals(types.iloc[1], np.int32) + @unittest.skipIf(have_pandas, "Required Pandas was found.") def test_to_pandas_required_pandas_not_found(self): with QuietTest(self.sc): diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index ddb8283cafa85..52ae74df5d4f2 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -17,9 +17,11 @@ import datetime import sys +from itertools import chain +import re from pyspark.sql import Row -from pyspark.sql.functions import udf, input_file_name +from pyspark.sql.functions import udf, input_file_name, col, percentile_approx, lit from pyspark.testing.sqlutils import ReusedSQLTestCase @@ -337,6 +339,29 @@ def test_overlay(self): self.assertListEqual(actual, expected) + def test_percentile_approx(self): + actual = list(chain.from_iterable([ + re.findall("(percentile_approx\\(.*\\))", str(x)) for x in [ + percentile_approx(col("foo"), lit(0.5)), + percentile_approx(col("bar"), 0.25, 42), + percentile_approx(col("bar"), [0.25, 0.5, 0.75]), + percentile_approx(col("foo"), (0.05, 0.95), 100), + percentile_approx("foo", 0.5), + percentile_approx("bar", [0.1, 0.9], lit(10)), + ] + ])) + + expected = [ + "percentile_approx(foo, 0.5, 10000)", + "percentile_approx(bar, 0.25, 42)", + "percentile_approx(bar, array(0.25, 0.5, 0.75), 10000)", + "percentile_approx(foo, array(0.05, 0.95), 100)", + "percentile_approx(foo, 0.5, 10000)", + "percentile_approx(bar, array(0.1, 0.9), 10)" + ] + + self.assertListEqual(actual, expected) + def test_higher_order_function_failures(self): from pyspark.sql.functions import col, exists, transform diff --git a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py index 21679785a769e..224c8ce97f559 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py @@ -491,6 +491,23 @@ def max_udf(v): agg2 = self.spark.sql("select max_udf(id) from table") assert_frame_equal(agg1.toPandas(), agg2.toPandas()) + def test_no_predicate_pushdown_through(self): + # SPARK-30921: We should not pushdown predicates of PythonUDFs through Aggregate. + import numpy as np + + @pandas_udf('float', PandasUDFType.GROUPED_AGG) + def mean(x): + return np.mean(x) + + df = self.spark.createDataFrame([ + Row(id=1, foo=42), Row(id=2, foo=1), Row(id=2, foo=2) + ]) + + agg = df.groupBy('id').agg(mean('foo').alias("mean")) + filtered = agg.filter(agg['mean'] > 40.0) + + assert(filtered.collect()[0]["mean"] == 42.0) + if __name__ == "__main__": from pyspark.sql.tests.test_pandas_udf_grouped_agg import * diff --git a/python/pyspark/sql/tests/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/test_pandas_udf_scalar.py index b07de3c0b3cc9..7260e80e2cfca 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/test_pandas_udf_scalar.py @@ -445,8 +445,8 @@ def iter_udf_wong_output_size(it): with QuietTest(self.sc): with self.assertRaisesRegexp( Exception, - "The number of output rows of pandas iterator UDF should be " - "the same with input rows"): + "The length of output in Scalar iterator.*" + "the length of output was 1"): df.select(iter_udf_wong_output_size(col('id'))).collect() @pandas_udf(LongType(), PandasUDFType.SCALAR_ITER) @@ -461,7 +461,7 @@ def iter_udf_not_reading_all_input(it): with QuietTest(self.sc): with self.assertRaisesRegexp( Exception, - "SQL_SCALAR_PANDAS_ITER_UDF should exhaust the input iterator"): + "pandas iterator UDF should exhaust"): df1.select(iter_udf_not_reading_all_input(col('id'))).collect() def test_vectorized_udf_chained(self): diff --git a/python/pyspark/sql/tests/test_pandas_udf_typehints.py b/python/pyspark/sql/tests/test_pandas_udf_typehints.py index 7c83c78f108d9..2582080056864 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_typehints.py +++ b/python/pyspark/sql/tests/test_pandas_udf_typehints.py @@ -261,6 +261,48 @@ def test_group_agg_udf_type_hint(self): expected = df.groupby('id').agg(mean(df.v).alias('weighted_mean(v, 1.0)')).sort('id') assert_frame_equal(expected.toPandas(), actual.toPandas()) + def test_ignore_type_hint_in_group_apply_in_pandas(self): + df = self.spark.range(10) + exec( + "def pandas_plus_one(v: pd.DataFrame) -> pd.DataFrame:\n" + " return v + 1", + self.local) + + pandas_plus_one = self.local["pandas_plus_one"] + + actual = df.groupby('id').applyInPandas(pandas_plus_one, schema=df.schema).sort('id') + expected = df.selectExpr("id + 1 as id") + assert_frame_equal(expected.toPandas(), actual.toPandas()) + + def test_ignore_type_hint_in_cogroup_apply_in_pandas(self): + df = self.spark.range(10) + exec( + "def pandas_plus_one(left: pd.DataFrame, right: pd.DataFrame) -> pd.DataFrame:\n" + " return left + 1", + self.local) + + pandas_plus_one = self.local["pandas_plus_one"] + + actual = df.groupby('id').cogroup( + self.spark.range(10).groupby("id") + ).applyInPandas(pandas_plus_one, schema=df.schema).sort('id') + expected = df.selectExpr("id + 1 as id") + assert_frame_equal(expected.toPandas(), actual.toPandas()) + + def test_ignore_type_hint_in_map_in_pandas(self): + df = self.spark.range(10) + exec( + "from typing import Iterator\n" + "def pandas_plus_one(iter: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]:\n" + " return map(lambda v: v + 1, iter)", + self.local) + + pandas_plus_one = self.local["pandas_plus_one"] + + actual = df.mapInPandas(pandas_plus_one, schema=df.schema) + expected = df.selectExpr("id + 1 as id") + assert_frame_equal(expected.toPandas(), actual.toPandas()) + if __name__ == "__main__": from pyspark.sql.tests.test_pandas_udf_typehints import * diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 7d62bf1783931..a9a32ed150c0b 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -341,7 +341,7 @@ def read_udfs(pickleSer, infile, eval_type): pickleSer, infile, eval_type, runner_conf, udf_index=0) def func(_, iterator): - num_input_rows = [0] + num_input_rows = [0] # TODO(SPARK-29909): Use nonlocal after we drop Python 2. def map_batch(batch): udf_args = [batch[offset] for offset in arg_offsets] @@ -357,8 +357,13 @@ def map_batch(batch): num_output_rows = 0 for result_batch, result_type in result_iter: num_output_rows += len(result_batch) + # This assert is for Scalar Iterator UDF to fail fast. + # The length of the entire input can only be explicitly known + # by consuming the input iterator in user side. Therefore, + # it's very unlikely the output length is higher than + # input length. assert is_map_iter or num_output_rows <= num_input_rows[0], \ - "Pandas MAP_ITER UDF outputted more rows than input rows." + "Pandas SCALAR_ITER UDF outputted more rows than input rows." yield (result_batch, result_type) if is_scalar_iter: @@ -367,14 +372,14 @@ def map_batch(batch): except StopIteration: pass else: - raise RuntimeError("SQL_SCALAR_PANDAS_ITER_UDF should exhaust the input " + raise RuntimeError("pandas iterator UDF should exhaust the input " "iterator.") - if is_scalar_iter and num_output_rows != num_input_rows[0]: - raise RuntimeError("The number of output rows of pandas iterator UDF should be " - "the same with input rows. The input rows number is %d but the " - "output rows number is %d." % - (num_input_rows[0], num_output_rows)) + if num_output_rows != num_input_rows[0]: + raise RuntimeError( + "The length of output in Scalar iterator pandas UDF should be " + "the same with the input's; however, the length of output was %d and the " + "length of input was %d." % (num_output_rows, num_input_rows[0])) # profiling is not supported for UDF return func, None, ser, ser diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 18793defee1be..b527816015c63 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -29,6 +29,7 @@ Spark Project Kubernetes kubernetes + 4.7.1 diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index f42f3415baa15..a6fc8519108c6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -77,7 +77,8 @@ private[spark] class KubernetesDriverConf( val appId: String, val mainAppResource: MainAppResource, val mainClass: String, - val appArgs: Array[String]) + val appArgs: Array[String], + val proxyUser: Option[String]) extends KubernetesConf(sparkConf) { override val resourceNamePrefix: String = { @@ -193,11 +194,18 @@ private[spark] object KubernetesConf { appId: String, mainAppResource: MainAppResource, mainClass: String, - appArgs: Array[String]): KubernetesDriverConf = { + appArgs: Array[String], + proxyUser: Option[String]): KubernetesDriverConf = { // Parse executor volumes in order to verify configuration before the driver pod is created. KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX) - new KubernetesDriverConf(sparkConf.clone(), appId, mainAppResource, mainClass, appArgs) + new KubernetesDriverConf( + sparkConf.clone(), + appId, + mainAppResource, + mainClass, + appArgs, + proxyUser) } def createExecutorConf( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala index 7faf0d75bdc75..ebe44855f1d0d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala @@ -92,8 +92,14 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesDriverConf) } else { resource } + var proxyUserArgs = Seq[String]() + if (!conf.proxyUser.isEmpty) { + proxyUserArgs = proxyUserArgs :+ "--proxy-user" + proxyUserArgs = proxyUserArgs :+ conf.proxyUser.get + } new ContainerBuilder(pod.container) .addToArgs("driver") + .addToArgs(proxyUserArgs: _*) .addToArgs("--properties-file", SPARK_CONF_PATH) .addToArgs("--class", conf.mainClass) .addToArgs(resolvedResource) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 8e5532d70b5da..b4155fed8aa24 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -43,7 +43,8 @@ import org.apache.spark.util.Utils private[spark] case class ClientArguments( mainAppResource: MainAppResource, mainClass: String, - driverArgs: Array[String]) + driverArgs: Array[String], + proxyUser: Option[String]) private[spark] object ClientArguments { @@ -51,6 +52,7 @@ private[spark] object ClientArguments { var mainAppResource: MainAppResource = JavaMainAppResource(None) var mainClass: Option[String] = None val driverArgs = mutable.ArrayBuffer.empty[String] + var proxyUser: Option[String] = None args.sliding(2, 2).toList.foreach { case Array("--primary-java-resource", primaryJavaResource: String) => @@ -63,6 +65,8 @@ private[spark] object ClientArguments { mainClass = Some(clazz) case Array("--arg", arg: String) => driverArgs += arg + case Array("--proxy-user", user: String) => + proxyUser = Some(user) case other => val invalid = other.mkString(" ") throw new RuntimeException(s"Unknown arguments: $invalid") @@ -73,7 +77,8 @@ private[spark] object ClientArguments { ClientArguments( mainAppResource, mainClass.get, - driverArgs.toArray) + driverArgs.toArray, + proxyUser) } } @@ -199,7 +204,8 @@ private[spark] class KubernetesClientApplication extends SparkApplication { kubernetesAppId, clientArguments.mainAppResource, clientArguments.mainClass, - clientArguments.driverArgs) + clientArguments.driverArgs, + clientArguments.proxyUser) // The master URL has been checked for validity already in SparkSubmit. // We just need to get rid of the "k8s://" prefix here. val master = KubernetesUtils.parseMasterUrl(sparkConf.get("spark.master")) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala index 83daddf714489..34fca296ddf18 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodStates.scala @@ -34,4 +34,6 @@ case class PodFailed(pod: Pod) extends FinalPodState case class PodDeleted(pod: Pod) extends FinalPodState +case class PodTerminating(pod: Pod) extends FinalPodState + case class PodUnknown(pod: Pod) extends ExecutorPodState diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala index 435a5f1461c92..30030ab539048 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala @@ -64,6 +64,8 @@ object ExecutorPodsSnapshot extends Logging { PodFailed(pod) case "succeeded" => PodSucceeded(pod) + case "terminating" => + PodTerminating(pod) case _ => logWarning(s"Received unknown phase $phase for executor pod with name" + s" ${pod.getMetadata.getName} in namespace ${pod.getMetadata.getNamespace}") @@ -72,5 +74,12 @@ object ExecutorPodsSnapshot extends Logging { } } - private def isDeleted(pod: Pod): Boolean = pod.getMetadata.getDeletionTimestamp != null + private def isDeleted(pod: Pod): Boolean = { + (pod.getMetadata.getDeletionTimestamp != null && + ( + pod.getStatus == null || + pod.getStatus.getPhase == null || + pod.getStatus.getPhase.toLowerCase(Locale.ROOT) != "terminating" + )) + } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index 5591974c564e8..1ca4dbc8674fb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -75,7 +75,8 @@ class KubernetesConfSuite extends SparkFunSuite { KubernetesTestConf.APP_ID, JavaMainAppResource(None), KubernetesTestConf.MAIN_CLASS, - APP_ARGS) + APP_ARGS, + None) assert(conf.labels === Map( SPARK_APP_ID_LABEL -> KubernetesTestConf.APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) ++ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala index ee830a91f3272..e8cdf63bb7762 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala @@ -50,7 +50,8 @@ object KubernetesTestConf { annotations: Map[String, String] = Map.empty, secretEnvNamesToKeyRefs: Map[String, String] = Map.empty, secretNamesToMountPaths: Map[String, String] = Map.empty, - volumes: Seq[KubernetesVolumeSpec] = Seq.empty): KubernetesDriverConf = { + volumes: Seq[KubernetesVolumeSpec] = Seq.empty, + proxyUser: Option[String] = None): KubernetesDriverConf = { val conf = sparkConf.clone() resourceNamePrefix.foreach { prefix => @@ -63,7 +64,7 @@ object KubernetesTestConf { setPrefixedConfigs(conf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX, secretEnvNamesToKeyRefs) setVolumeSpecs(conf, KUBERNETES_DRIVER_VOLUMES_PREFIX, volumes) - new KubernetesDriverConf(conf, appId, mainAppResource, mainClass, appArgs) + new KubernetesDriverConf(conf, appId, mainAppResource, mainClass, appArgs, proxyUser) } // scalastyle:on argcount diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala index de80c5614cbfc..829943f16beac 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala @@ -76,14 +76,64 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { mainResource, "5", "7", "9")) } + test("SPARK-25355: java resource args with proxy-user") { + val mainResource = "local:/main.jar" + val spec = applyFeatureStep( + JavaMainAppResource(Some(mainResource)), + appArgs = Array("5", "7"), + proxyUser = Some("test.user")) + assert(spec.pod.container.getArgs.asScala === List( + "driver", + "--proxy-user", "test.user", + "--properties-file", SPARK_CONF_PATH, + "--class", KubernetesTestConf.MAIN_CLASS, + mainResource, "5", "7")) + } + + test("SPARK-25355: python resource args with proxy-user") { + val mainResource = "local:/main.py" + val sparkConf = new SparkConf(false) + .set(PYSPARK_MAJOR_PYTHON_VERSION, "2") + val spec = applyFeatureStep( + PythonMainAppResource(mainResource), + conf = sparkConf, + appArgs = Array("5", "7", "9"), + proxyUser = Some("test.user")) + + assert(spec.pod.container.getArgs.asScala === List( + "driver", + "--proxy-user", "test.user", + "--properties-file", SPARK_CONF_PATH, + "--class", KubernetesTestConf.MAIN_CLASS, + mainResource, "5", "7", "9")) + } + + test("SPARK-25355: R resource args with proxy-user") { + val mainResource = "local:/main.R" + + val spec = applyFeatureStep( + RMainAppResource(mainResource), + appArgs = Array("5", "7", "9"), + proxyUser = Some("test.user")) + + assert(spec.pod.container.getArgs.asScala === List( + "driver", + "--proxy-user", "test.user", + "--properties-file", SPARK_CONF_PATH, + "--class", KubernetesTestConf.MAIN_CLASS, + mainResource, "5", "7", "9")) + } + private def applyFeatureStep( resource: MainAppResource, conf: SparkConf = new SparkConf(false), - appArgs: Array[String] = Array()): KubernetesDriverSpec = { + appArgs: Array[String] = Array(), + proxyUser: Option[String] = None): KubernetesDriverSpec = { val kubernetesConf = KubernetesTestConf.createDriverConf( sparkConf = conf, mainAppResource = resource, - appArgs = appArgs) + appArgs = appArgs, + proxyUser = proxyUser) val step = new DriverCommandFeatureStep(kubernetesConf) val pod = step.configurePod(SparkPod.initialPod()) val props = step.getAdditionalPodSystemProperties() diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index cc65a7da12eef..b722123195dc2 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -28,6 +28,7 @@ ARG spark_uid=185 # docker build -t spark:latest -f kubernetes/dockerfiles/spark/Dockerfile . RUN set -ex && \ + sed -i 's/http:/https:/g' /etc/apt/sources.list && \ apt-get update && \ ln -s /lib /lib64 && \ apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps && \ diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 05ab782caecae..813a70c6e7ec3 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -58,8 +58,8 @@ fi # If HADOOP_HOME is set and SPARK_DIST_CLASSPATH is not set, set it here so Hadoop jars are available to the executor. # It does not set SPARK_DIST_CLASSPATH if already set, to avoid overriding customizations of this value from elsewhere e.g. Docker/K8s. -if [ -n ${HADOOP_HOME} ] && [ -z ${SPARK_DIST_CLASSPATH} ]; then - export SPARK_DIST_CLASSPATH=$($HADOOP_HOME/bin/hadoop classpath) +if [ -n "${HADOOP_HOME}" ] && [ -z "${SPARK_DIST_CLASSPATH}" ]; then + export SPARK_DIST_CLASSPATH="$($HADOOP_HOME/bin/hadoop classpath)" fi if ! [ -z ${HADOOP_CONF_DIR+x} ]; then diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index a522e87c4a4f8..7a889c427b41e 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -29,7 +29,7 @@ 1.3.0 1.4.0 - 4.6.4 + 4.7.1 3.2.2 1.0 kubernetes-integration-tests diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala index 367cff62cd493..2d90c06e36390 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala @@ -35,17 +35,16 @@ import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.Minikube private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => import KubernetesSuite.k8sTestTag - val cName = "ceph-nano" + val cName = "minio" val svcName = s"$cName-s3" - val bucket = "spark" - - private def getCephContainer(): Container = { - val envVars = Map ( "NETWORK_AUTO_DETECT" -> "4", - "RGW_FRONTEND_PORT" -> "8000", - "SREE_PORT" -> "5001", - "CEPH_DEMO_UID" -> "nano", - "CEPH_DAEMON" -> "demo", - "DEBUG" -> "verbose" + val BUCKET = "spark" + val ACCESS_KEY = "minio" + val SECRET_KEY = "miniostorage" + + private def getMinioContainer(): Container = { + val envVars = Map ( + "MINIO_ACCESS_KEY" -> ACCESS_KEY, + "MINIO_SECRET_KEY" -> SECRET_KEY ).map( envV => new EnvVarBuilder() .withName(envV._1) @@ -59,13 +58,14 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => ).asJava new ContainerBuilder() - .withImage("ceph/daemon:v4.0.3-stable-4.0-nautilus-centos-7-x86_64") + .withImage("minio/minio:latest") .withImagePullPolicy("Always") .withName(cName) + .withArgs("server", "/data") .withPorts(new ContainerPortBuilder() .withName(svcName) .withProtocol("TCP") - .withContainerPort(8000) + .withContainerPort(9000) .build() ) .withResources(new ResourceRequirementsBuilder() @@ -77,10 +77,9 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => .build() } - // Based on https://github.com/ceph/cn - private def setupCephStorage(): Unit = { - val labels = Map("app" -> "ceph", "daemon" -> "nano").asJava - val cephService = new ServiceBuilder() + private def setupMinioStorage(): Unit = { + val labels = Map("app" -> "minio").asJava + val minioService = new ServiceBuilder() .withNewMetadata() .withName(svcName) .withLabels(labels) @@ -88,9 +87,9 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => .withNewSpec() .withPorts(new ServicePortBuilder() .withName("https") - .withPort(8000) + .withPort(9000) .withProtocol("TCP") - .withTargetPort(new IntOrString(8000)) + .withTargetPort(new IntOrString(9000)) .build() ) .withType("NodePort") @@ -98,7 +97,7 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => .endSpec() .build() - val cephStatefulSet = new StatefulSetBuilder() + val minioStatefulSet = new StatefulSetBuilder() .withNewMetadata() .withName(cName) .withLabels(labels) @@ -106,7 +105,7 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => .withNewSpec() .withReplicas(1) .withNewSelector() - .withMatchLabels(Map("app" -> "ceph").asJava) + .withMatchLabels(Map("app" -> "minio").asJava) .endSelector() .withServiceName(cName) .withNewTemplate() @@ -115,7 +114,7 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => .withLabels(labels) .endMetadata() .withNewSpec() - .withContainers(getCephContainer()) + .withContainers(getMinioContainer()) .endSpec() .endTemplate() .endSpec() @@ -124,16 +123,16 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => kubernetesTestComponents .kubernetesClient .services() - .create(cephService) + .create(minioService) kubernetesTestComponents .kubernetesClient .apps() .statefulSets() - .create(cephStatefulSet) + .create(minioStatefulSet) } - private def deleteCephStorage(): Unit = { + private def deleteMinioStorage(): Unit = { kubernetesTestComponents .kubernetesClient .apps() @@ -151,47 +150,30 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => test("Launcher client dependencies", k8sTestTag, MinikubeTag) { val fileName = Utils.createTempFile(FILE_CONTENTS, HOST_PATH) try { - setupCephStorage() - val cephUrlStr = getServiceUrl(svcName) - val cephUrl = new URL(cephUrlStr) - val cephHost = cephUrl.getHost - val cephPort = cephUrl.getPort + setupMinioStorage() + val minioUrlStr = getServiceUrl(svcName) + val minioUrl = new URL(minioUrlStr) + val minioHost = minioUrl.getHost + val minioPort = minioUrl.getPort val examplesJar = Utils.getExamplesJarAbsolutePath(sparkHomeDir) - val (accessKey, secretKey) = getCephCredentials() sparkAppConf - .set("spark.hadoop.fs.s3a.access.key", accessKey) - .set("spark.hadoop.fs.s3a.secret.key", secretKey) + .set("spark.hadoop.fs.s3a.access.key", ACCESS_KEY) + .set("spark.hadoop.fs.s3a.secret.key", SECRET_KEY) .set("spark.hadoop.fs.s3a.connection.ssl.enabled", "false") - .set("spark.hadoop.fs.s3a.endpoint", s"$cephHost:$cephPort") - .set("spark.kubernetes.file.upload.path", s"s3a://$bucket") + .set("spark.hadoop.fs.s3a.endpoint", s"$minioHost:$minioPort") + .set("spark.kubernetes.file.upload.path", s"s3a://$BUCKET") .set("spark.files", s"$HOST_PATH/$fileName") .set("spark.hadoop.fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem") .set("spark.jars.packages", "com.amazonaws:aws-java-sdk:" + "1.7.4,org.apache.hadoop:hadoop-aws:2.7.6") .set("spark.driver.extraJavaOptions", "-Divy.cache.dir=/tmp -Divy.home=/tmp") - createS3Bucket(accessKey, secretKey, cephUrlStr) + createS3Bucket(ACCESS_KEY, SECRET_KEY, minioUrlStr) runSparkRemoteCheckAndVerifyCompletion(appResource = examplesJar, appArgs = Array(fileName), timeout = Option(DEPS_TIMEOUT)) } finally { // make sure this always runs - deleteCephStorage() - } - } - - // There isn't a cleaner way to get the credentials - // when ceph-nano runs on k8s - private def getCephCredentials(): (String, String) = { - Eventually.eventually(TIMEOUT, INTERVAL) { - val cephPod = kubernetesTestComponents - .kubernetesClient - .pods() - .withName(s"$cName-0") - .get() - implicit val podName: String = cephPod.getMetadata.getName - implicit val components: KubernetesTestComponents = kubernetesTestComponents - val contents = Utils.executeCommand("cat", "/nano_user_details") - (extractS3Key(contents, "access_key"), extractS3Key(contents, "secret_key")) + deleteMinioStorage() } } @@ -211,10 +193,10 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => val credentials = new BasicAWSCredentials(accessKey, secretKey) val s3client = new AmazonS3Client(credentials) s3client.setEndpoint(endPoint) - s3client.createBucket(bucket) + s3client.createBucket(BUCKET) } catch { case e: Exception => - throw new SparkException(s"Failed to create bucket $bucket.", e) + throw new SparkException(s"Failed to create bucket $BUCKET.", e) } } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index eaaf67db69201..4de7e70c1f409 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -61,15 +61,15 @@ class KubernetesSuite extends SparkFunSuite protected var appLocator: String = _ // Default memory limit is 1024M + 384M (minimum overhead constant) - private val baseMemory = s"${1024 + 384}Mi" + private val baseMemory = s"${1024 + 384}" protected val memOverheadConstant = 0.8 - private val standardNonJVMMemory = s"${(1024 + 0.4*1024).toInt}Mi" + private val standardNonJVMMemory = s"${(1024 + 0.4*1024).toInt}" protected val additionalMemory = 200 // 209715200 is 200Mi protected val additionalMemoryInBytes = 209715200 - private val extraDriverTotalMemory = s"${(1024 + memOverheadConstant*1024).toInt}Mi" + private val extraDriverTotalMemory = s"${(1024 + memOverheadConstant*1024).toInt}" private val extraExecTotalMemory = - s"${(1024 + memOverheadConstant*1024 + additionalMemory).toInt}Mi" + s"${(1024 + memOverheadConstant*1024 + additionalMemory).toInt}" /** * Build the image ref for the given image name, taking the repo and tag from the diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala index 4b4dff93f5742..86f8cddcef7da 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala @@ -56,7 +56,8 @@ private[spark] trait PVTestsSuite { k8sSuite: KubernetesSuite => .withMatchExpressions(new NodeSelectorRequirementBuilder() .withKey("kubernetes.io/hostname") .withOperator("In") - .withValues("minikube", "docker-for-desktop", "docker-desktop").build()).build()) + .withValues("minikube", "m01", "docker-for-desktop", "docker-desktop") + .build()).build()) .endRequired() .endNodeAffinity() .endSpec() diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala index e2a99148dd799..f1e3fcab7e6af 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala @@ -31,6 +31,7 @@ import org.apache.spark.{SparkContext, SparkException, TaskState} import org.apache.spark.deploy.mesos.{config => mesosConfig} import org.apache.spark.executor.MesosExecutorBackend import org.apache.spark.internal.config +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils @@ -457,7 +458,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( super.applicationId } - override def maxNumConcurrentTasks(): Int = { + override def maxNumConcurrentTasks(rp: ResourceProfile): Int = { // TODO SPARK-25074 support this method for MesosFineGrainedSchedulerBackend 0 } diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala index a5bd34888a0a6..36ed84858dbfb 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala @@ -303,7 +303,8 @@ class MesosFineGrainedSchedulerBackendSuite mesosOffers2.add(createOffer(1, minMem, minCpu)) reset(taskScheduler) reset(driver) - when(taskScheduler.resourceOffers(any(classOf[IndexedSeq[WorkerOffer]]))).thenReturn(Seq(Seq())) + when(taskScheduler.resourceOffers(any(classOf[IndexedSeq[WorkerOffer]]), any[Boolean])) + .thenReturn(Seq(Seq())) when(taskScheduler.CPUS_PER_TASK).thenReturn(2) when(driver.declineOffer(mesosOffers2.get(0).getId)).thenReturn(Status.valueOf(1)) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index a5576a73f1e80..4ae7b737556b6 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -80,7 +80,7 @@ grammar SqlBase; } singleStatement - : statement EOF + : statement ';'* EOF ; singleExpression @@ -174,7 +174,7 @@ statement SET TBLPROPERTIES tablePropertyList #setTableProperties | ALTER (TABLE | VIEW) multipartIdentifier UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties - |ALTER TABLE table=multipartIdentifier + | ALTER TABLE table=multipartIdentifier (ALTER | CHANGE) COLUMN? column=multipartIdentifier alterColumnAction? #alterTableAlterColumn | ALTER TABLE table=multipartIdentifier partitionSpec? @@ -223,6 +223,8 @@ statement ('(' key=tablePropertyKey ')')? #showTblProperties | SHOW COLUMNS (FROM | IN) table=multipartIdentifier ((FROM | IN) ns=multipartIdentifier)? #showColumns + | SHOW VIEWS ((FROM | IN) multipartIdentifier)? + (LIKE? pattern=STRING)? #showViews | SHOW PARTITIONS multipartIdentifier partitionSpec? #showPartitions | SHOW identifier? FUNCTIONS (LIKE? (multipartIdentifier | pattern=STRING))? #showFunctions @@ -1190,6 +1192,7 @@ ansiNonReserved | USE | VALUES | VIEW + | VIEWS | WINDOW ; @@ -1454,6 +1457,7 @@ nonReserved | USER | VALUES | VIEW + | VIEWS | WHEN | WHERE | WINDOW @@ -1715,6 +1719,7 @@ USER: 'USER'; USING: 'USING'; VALUES: 'VALUES'; VIEW: 'VIEW'; +VIEWS: 'VIEWS'; WHEN: 'WHEN'; WHERE: 'WHERE'; WINDOW: 'WINDOW'; diff --git a/sql/catalyst/src/main/resources/gregorian-julian-rebase-micros.json b/sql/catalyst/src/main/resources/gregorian-julian-rebase-micros.json new file mode 100644 index 0000000000000..050aa02918fc3 --- /dev/null +++ b/sql/catalyst/src/main/resources/gregorian-julian-rebase-micros.json @@ -0,0 +1,2385 @@ +[ { + "tz" : "Africa/Abidjan", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Accra", + "switches" : [ -62135596748, -59006361548, -55850687948, -52695014348, -46383580748, -43227907148, -40072233548, -33760799948, -30605126348, -27449452748, -21138019148, -17982345548, -14826671948, -12219292748, -2208988800 ], + "diffs" : [ -172852, -86452, -52, 86348, 172748, 259148, 345548, 431948, 518348, 604748, 691148, 777548, 863948, -52, 0 ] +}, { + "tz" : "Africa/Addis_Ababa", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Africa/Algiers", + "switches" : [ -62135597532, -59006362332, -55850688732, -52695015132, -46383581532, -43227907932, -40072234332, -33760800732, -30605127132, -27449453532, -21138019932, -17982346332, -14826672732, -12219293532, -2486679072, -2208988800 ], + "diffs" : [ -175668, -89268, -2868, 83532, 169932, 256332, 342732, 429132, 515532, 601932, 688332, 774732, 861132, -2868, -3039, 0 ] +}, { + "tz" : "Africa/Asmara", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Africa/Asmera", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Africa/Bamako", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Bangui", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Banjul", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Bissau", + "switches" : [ -62135593060, -59006357860, -55850684260, -52695010660, -46383577060, -43227903460, -40072229860, -33760796260, -30605122660, -27449449060, -21138015460, -17982341860, -14826668260, -12219289060, -2208988800 ], + "diffs" : [ -176540, -90140, -3740, 82660, 169060, 255460, 341860, 428260, 514660, 601060, 687460, 773860, 860260, -3740, 0 ] +}, { + "tz" : "Africa/Blantyre", + "switches" : [ -62135604620, -59006369420, -55850695820, -52695022220, -46383588620, -43227915020, -40072241420, -33760807820, -30605134220, -27449460620, -21138027020, -17982353420, -14826679820, -12219300620, -2208988800 ], + "diffs" : [ -172180, -85780, 620, 87020, 173420, 259820, 346220, 432620, 519020, 605420, 691820, 778220, 864620, 620, 0 ] +}, { + "tz" : "Africa/Brazzaville", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Bujumbura", + "switches" : [ -62135604620, -59006369420, -55850695820, -52695022220, -46383588620, -43227915020, -40072241420, -33760807820, -30605134220, -27449460620, -21138027020, -17982353420, -14826679820, -12219300620, -2208988800 ], + "diffs" : [ -172180, -85780, 620, 87020, 173420, 259820, 346220, 432620, 519020, 605420, 691820, 778220, 864620, 620, 0 ] +}, { + "tz" : "Africa/Cairo", + "switches" : [ -62135604309, -59006369109, -55850695509, -52695021909, -46383588309, -43227914709, -40072241109, -33760807509, -30605133909, -27449460309, -21138026709, -17982353109, -14826679509, -12219300309, -2208988800 ], + "diffs" : [ -172491, -86091, 309, 86709, 173109, 259509, 345909, 432309, 518709, 605109, 691509, 777909, 864309, 309, 0 ] +}, { + "tz" : "Africa/Ceuta", + "switches" : [ -62135595524, -59006360324, -55850686724, -52695013124, -46383579524, -43227905924, -40072232324, -33760798724, -30605125124, -27449451524, -21138017924, -17982344324, -14826670724, -12219291524, -2208988800 ], + "diffs" : [ -177676, -91276, -4876, 81524, 167924, 254324, 340724, 427124, 513524, 599924, 686324, 772724, 859124, -4876, 0 ] +}, { + "tz" : "Africa/Conakry", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Dakar", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Dar_es_Salaam", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Africa/Djibouti", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Africa/Douala", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Freetown", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Gaborone", + "switches" : [ -62135604620, -59006369420, -55850695820, -52695022220, -46383588620, -43227915020, -40072241420, -33760807820, -30605134220, -27449460620, -21138027020, -17982353420, -14826679820, -12219300620, -2208988800 ], + "diffs" : [ -172180, -85780, 620, 87020, 173420, 259820, 346220, 432620, 519020, 605420, 691820, 778220, 864620, 620, 0 ] +}, { + "tz" : "Africa/Harare", + "switches" : [ -62135604620, -59006369420, -55850695820, -52695022220, -46383588620, -43227915020, -40072241420, -33760807820, -30605134220, -27449460620, -21138027020, -17982353420, -14826679820, -12219300620, -2208988800 ], + "diffs" : [ -172180, -85780, 620, 87020, 173420, 259820, 346220, 432620, 519020, 605420, 691820, 778220, 864620, 620, 0 ] +}, { + "tz" : "Africa/Johannesburg", + "switches" : [ -62135603520, -59006368320, -55850694720, -52695021120, -46383587520, -43227913920, -40072240320, -33760806720, -30605133120, -27449459520, -21138025920, -17982352320, -14826678720, -12219299520, -2458173120, -2208988800 ], + "diffs" : [ -173280, -86880, -480, 85920, 172320, 258720, 345120, 431520, 517920, 604320, 690720, 777120, 863520, -480, -1800, 0 ] +}, { + "tz" : "Africa/Juba", + "switches" : [ -62135604388, -59006369188, -55850695588, -52695021988, -46383588388, -43227914788, -40072241188, -33760807588, -30605133988, -27449460388, -21138026788, -17982353188, -14826679588, -12219300388, -2208988800 ], + "diffs" : [ -176012, -89612, -3212, 83188, 169588, 255988, 342388, 428788, 515188, 601588, 687988, 774388, 860788, -3212, 0 ] +}, { + "tz" : "Africa/Kampala", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Africa/Khartoum", + "switches" : [ -62135604608, -59006369408, -55850695808, -52695022208, -46383588608, -43227915008, -40072241408, -33760807808, -30605134208, -27449460608, -21138027008, -17982353408, -14826679808, -12219300608, -2208988800 ], + "diffs" : [ -172192, -85792, 608, 87008, 173408, 259808, 346208, 432608, 519008, 605408, 691808, 778208, 864608, 608, 0 ] +}, { + "tz" : "Africa/Kigali", + "switches" : [ -62135604620, -59006369420, -55850695820, -52695022220, -46383588620, -43227915020, -40072241420, -33760807820, -30605134220, -27449460620, -21138027020, -17982353420, -14826679820, -12219300620, -2208988800 ], + "diffs" : [ -172180, -85780, 620, 87020, 173420, 259820, 346220, 432620, 519020, 605420, 691820, 778220, 864620, 620, 0 ] +}, { + "tz" : "Africa/Kinshasa", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Lagos", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Libreville", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Lome", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Luanda", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Lubumbashi", + "switches" : [ -62135604620, -59006369420, -55850695820, -52695022220, -46383588620, -43227915020, -40072241420, -33760807820, -30605134220, -27449460620, -21138027020, -17982353420, -14826679820, -12219300620, -2208988800 ], + "diffs" : [ -172180, -85780, 620, 87020, 173420, 259820, 346220, 432620, 519020, 605420, 691820, 778220, 864620, 620, 0 ] +}, { + "tz" : "Africa/Lusaka", + "switches" : [ -62135604620, -59006369420, -55850695820, -52695022220, -46383588620, -43227915020, -40072241420, -33760807820, -30605134220, -27449460620, -21138027020, -17982353420, -14826679820, -12219300620, -2208988800 ], + "diffs" : [ -172180, -85780, 620, 87020, 173420, 259820, 346220, 432620, 519020, 605420, 691820, 778220, 864620, 620, 0 ] +}, { + "tz" : "Africa/Malabo", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Maputo", + "switches" : [ -62135604620, -59006369420, -55850695820, -52695022220, -46383588620, -43227915020, -40072241420, -33760807820, -30605134220, -27449460620, -21138027020, -17982353420, -14826679820, -12219300620, -2208988800 ], + "diffs" : [ -172180, -85780, 620, 87020, 173420, 259820, 346220, 432620, 519020, 605420, 691820, 778220, 864620, 620, 0 ] +}, { + "tz" : "Africa/Maseru", + "switches" : [ -62135603520, -59006368320, -55850694720, -52695021120, -46383587520, -43227913920, -40072240320, -33760806720, -30605133120, -27449459520, -21138025920, -17982352320, -14826678720, -12219299520, -2458173120, -2208988800 ], + "diffs" : [ -173280, -86880, -480, 85920, 172320, 258720, 345120, 431520, 517920, 604320, 690720, 777120, 863520, -480, -1800, 0 ] +}, { + "tz" : "Africa/Mbabane", + "switches" : [ -62135603520, -59006368320, -55850694720, -52695021120, -46383587520, -43227913920, -40072240320, -33760806720, -30605133120, -27449459520, -21138025920, -17982352320, -14826678720, -12219299520, -2458173120, -2208988800 ], + "diffs" : [ -173280, -86880, -480, 85920, 172320, 258720, 345120, 431520, 517920, 604320, 690720, 777120, 863520, -480, -1800, 0 ] +}, { + "tz" : "Africa/Mogadishu", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Africa/Monrovia", + "switches" : [ -62135594212, -59006359012, -55850685412, -52695011812, -46383578212, -43227904612, -40072231012, -33760797412, -30605123812, -27449450212, -21138016612, -17982343012, -14826669412, -12219290212, -2208988800 ], + "diffs" : [ -175388, -88988, -2588, 83812, 170212, 256612, 343012, 429412, 515812, 602212, 688612, 775012, 861412, -2588, 0 ] +}, { + "tz" : "Africa/Nairobi", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Africa/Ndjamena", + "switches" : [ -62135600412, -59006365212, -55850691612, -52695018012, -46383584412, -43227910812, -40072237212, -33760803612, -30605130012, -27449456412, -21138022812, -17982349212, -14826675612, -12219296412, -2208988800 ], + "diffs" : [ -172788, -86388, 12, 86412, 172812, 259212, 345612, 432012, 518412, 604812, 691212, 777612, 864012, 12, 0 ] +}, { + "tz" : "Africa/Niamey", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Nouakchott", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Ouagadougou", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Porto-Novo", + "switches" : [ -62135597616, -59006362416, -55850688816, -52695015216, -46383581616, -43227908016, -40072234416, -33760800816, -30605127216, -27449453616, -21138020016, -17982346416, -14826672816, -12219293616, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Africa/Sao_Tome", + "switches" : [ -62135598416, -59006363216, -55850689616, -52695016016, -46383582416, -43227908816, -40072235216, -33760801616, -30605128016, -27449454416, -21138020816, -17982347216, -14826673616, -12219294416, -2713912016, -2208988800 ], + "diffs" : [ -171184, -84784, 1616, 88016, 174416, 260816, 347216, 433616, 520016, 606416, 692816, 779216, 865616, 1616, -2205, 0 ] +}, { + "tz" : "Africa/Timbuktu", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Africa/Tripoli", + "switches" : [ -62135599964, -59006364764, -55850691164, -52695017564, -46383583964, -43227910364, -40072236764, -33760803164, -30605129564, -27449455964, -21138022364, -17982348764, -14826675164, -12219295964, -2208988800 ], + "diffs" : [ -176836, -90436, -4036, 82364, 168764, 255164, 341564, 427964, 514364, 600764, 687164, 773564, 859964, -4036, 0 ] +}, { + "tz" : "Africa/Tunis", + "switches" : [ -62135599244, -59006364044, -55850690444, -52695016844, -46383583244, -43227909644, -40072236044, -33760802444, -30605128844, -27449455244, -21138021644, -17982348044, -14826674444, -12219295244, -2797202444, -2208988800 ], + "diffs" : [ -173956, -87556, -1156, 85244, 171644, 258044, 344444, 430844, 517244, 603644, 690044, 776444, 862844, -1156, -3039, 0 ] +}, { + "tz" : "Africa/Windhoek", + "switches" : [ -62135600904, -59006365704, -55850692104, -52695018504, -46383584904, -43227911304, -40072237704, -33760804104, -30605130504, -27449456904, -21138023304, -17982349704, -14826676104, -12219296904, -2458170504, -2208988800 ], + "diffs" : [ -175896, -89496, -3096, 83304, 169704, 256104, 342504, 428904, 515304, 601704, 688104, 774504, 860904, -3096, -1800, 0 ] +}, { + "tz" : "America/Adak", + "switches" : [ -62135640802, -59006405602, -55850732002, -52695058402, -46383624802, -43227951202, -40072277602, -33760844002, -30605170402, -27449496802, -21138063202, -17982389602, -14826716002, -12219336802, -3225223727, -2208988800 ], + "diffs" : [ -92798, -6398, 80002, 166402, 252802, 339202, 425602, 512002, 598402, 684802, 771202, 857602, 944002, 80002, -6398, 0 ] +}, { + "tz" : "America/Anchorage", + "switches" : [ -62135647224, -59006412024, -55850738424, -52695064824, -46383631224, -43227957624, -40072284024, -33760850424, -30605176824, -27449503224, -21138069624, -17982396024, -14826722424, -12219343224, -3225223727, -2208988800 ], + "diffs" : [ -89976, -3576, 82824, 169224, 255624, 342024, 428424, 514824, 601224, 687624, 774024, 860424, 946824, 82824, -3576, 0 ] +}, { + "tz" : "America/Anguilla", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Antigua", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Araguaina", + "switches" : [ -62135585232, -59006350032, -55850676432, -52695002832, -46383569232, -43227895632, -40072222032, -33760788432, -30605114832, -27449441232, -21138007632, -17982334032, -14826660432, -12219281232, -2208988800 ], + "diffs" : [ -173568, -87168, -768, 85632, 172032, 258432, 344832, 431232, 517632, 604032, 690432, 776832, 863232, -768, 0 ] +}, { + "tz" : "America/Argentina/Buenos_Aires", + "switches" : [ -62135582772, -59006347572, -55850673972, -52695000372, -46383566772, -43227893172, -40072219572, -33760785972, -30605112372, -27449438772, -21138005172, -17982331572, -14826657972, -12219278772, -2372097972, -2208988800 ], + "diffs" : [ -176028, -89628, -3228, 83172, 169572, 255972, 342372, 428772, 515172, 601572, 687972, 774372, 860772, -3228, -4608, 0 ] +}, { + "tz" : "America/Argentina/Catamarca", + "switches" : [ -62135581012, -59006345812, -55850672212, -52694998612, -46383565012, -43227891412, -40072217812, -33760784212, -30605110612, -27449437012, -21138003412, -17982329812, -14826656212, -12219277012, -2372096212, -2208988800 ], + "diffs" : [ -177788, -91388, -4988, 81412, 167812, 254212, 340612, 427012, 513412, 599812, 686212, 772612, 859012, -4988, -4608, 0 ] +}, { + "tz" : "America/Argentina/ComodRivadavia", + "switches" : [ -62135581012, -59006345812, -55850672212, -52694998612, -46383565012, -43227891412, -40072217812, -33760784212, -30605110612, -27449437012, -21138003412, -17982329812, -14826656212, -12219277012, -2372096212, -2208988800 ], + "diffs" : [ -177788, -91388, -4988, 81412, 167812, 254212, 340612, 427012, 513412, 599812, 686212, 772612, 859012, -4988, -4608, 0 ] +}, { + "tz" : "America/Argentina/Cordoba", + "switches" : [ -62135581392, -59006346192, -55850672592, -52694998992, -46383565392, -43227891792, -40072218192, -33760784592, -30605110992, -27449437392, -21138003792, -17982330192, -14826656592, -12219277392, -2208988800 ], + "diffs" : [ -177408, -91008, -4608, 81792, 168192, 254592, 340992, 427392, 513792, 600192, 686592, 772992, 859392, -4608, 0 ] +}, { + "tz" : "America/Argentina/Jujuy", + "switches" : [ -62135581128, -59006345928, -55850672328, -52694998728, -46383565128, -43227891528, -40072217928, -33760784328, -30605110728, -27449437128, -21138003528, -17982329928, -14826656328, -12219277128, -2372096328, -2208988800 ], + "diffs" : [ -177672, -91272, -4872, 81528, 167928, 254328, 340728, 427128, 513528, 599928, 686328, 772728, 859128, -4872, -4608, 0 ] +}, { + "tz" : "America/Argentina/La_Rioja", + "switches" : [ -62135580756, -59006345556, -55850671956, -52694998356, -46383564756, -43227891156, -40072217556, -33760783956, -30605110356, -27449436756, -21138003156, -17982329556, -14826655956, -12219276756, -2372095956, -2208988800 ], + "diffs" : [ -178044, -91644, -5244, 81156, 167556, 253956, 340356, 426756, 513156, 599556, 685956, 772356, 858756, -5244, -4608, 0 ] +}, { + "tz" : "America/Argentina/Mendoza", + "switches" : [ -62135580284, -59006345084, -55850671484, -52694997884, -46383564284, -43227890684, -40072217084, -33760783484, -30605109884, -27449436284, -21138002684, -17982329084, -14826655484, -12219276284, -2372095484, -2208988800 ], + "diffs" : [ -178516, -92116, -5716, 80684, 167084, 253484, 339884, 426284, 512684, 599084, 685484, 771884, 858284, -5716, -4608, 0 ] +}, { + "tz" : "America/Argentina/Rio_Gallegos", + "switches" : [ -62135580188, -59006344988, -55850671388, -52694997788, -46383564188, -43227890588, -40072216988, -33760783388, -30605109788, -27449436188, -21138002588, -17982328988, -14826655388, -12219276188, -2372095388, -2208988800 ], + "diffs" : [ -178612, -92212, -5812, 80588, 166988, 253388, 339788, 426188, 512588, 598988, 685388, 771788, 858188, -5812, -4608, 0 ] +}, { + "tz" : "America/Argentina/Salta", + "switches" : [ -62135581100, -59006345900, -55850672300, -52694998700, -46383565100, -43227891500, -40072217900, -33760784300, -30605110700, -27449437100, -21138003500, -17982329900, -14826656300, -12219277100, -2372096300, -2208988800 ], + "diffs" : [ -177700, -91300, -4900, 81500, 167900, 254300, 340700, 427100, 513500, 599900, 686300, 772700, 859100, -4900, -4608, 0 ] +}, { + "tz" : "America/Argentina/San_Juan", + "switches" : [ -62135580356, -59006345156, -55850671556, -52694997956, -46383564356, -43227890756, -40072217156, -33760783556, -30605109956, -27449436356, -21138002756, -17982329156, -14826655556, -12219276356, -2372095556, -2208988800 ], + "diffs" : [ -178444, -92044, -5644, 80756, 167156, 253556, 339956, 426356, 512756, 599156, 685556, 771956, 858356, -5644, -4608, 0 ] +}, { + "tz" : "America/Argentina/San_Luis", + "switches" : [ -62135580876, -59006345676, -55850672076, -52694998476, -46383564876, -43227891276, -40072217676, -33760784076, -30605110476, -27449436876, -21138003276, -17982329676, -14826656076, -12219276876, -2372096076, -2208988800 ], + "diffs" : [ -177924, -91524, -5124, 81276, 167676, 254076, 340476, 426876, 513276, 599676, 686076, 772476, 858876, -5124, -4608, 0 ] +}, { + "tz" : "America/Argentina/Tucuman", + "switches" : [ -62135581148, -59006345948, -55850672348, -52694998748, -46383565148, -43227891548, -40072217948, -33760784348, -30605110748, -27449437148, -21138003548, -17982329948, -14826656348, -12219277148, -2372096348, -2208988800 ], + "diffs" : [ -177652, -91252, -4852, 81548, 167948, 254348, 340748, 427148, 513548, 599948, 686348, 772748, 859148, -4852, -4608, 0 ] +}, { + "tz" : "America/Argentina/Ushuaia", + "switches" : [ -62135580408, -59006345208, -55850671608, -52694998008, -46383564408, -43227890808, -40072217208, -33760783608, -30605110008, -27449436408, -21138002808, -17982329208, -14826655608, -12219276408, -2372095608, -2208988800 ], + "diffs" : [ -178392, -91992, -5592, 80808, 167208, 253608, 340008, 426408, 512808, 599208, 685608, 772008, 858408, -5592, -4608, 0 ] +}, { + "tz" : "America/Aruba", + "switches" : [ -62135580253, -59006345053, -55850671453, -52694997853, -46383564253, -43227890653, -40072217053, -33760783453, -30605109853, -27449436253, -21138002653, -17982329053, -14826655453, -12219276253, -2208988800 ], + "diffs" : [ -174947, -88547, -2147, 84253, 170653, 257053, 343453, 429853, 516253, 602653, 689053, 775453, 861853, -2147, 0 ] +}, { + "tz" : "America/Asuncion", + "switches" : [ -62135582960, -59006347760, -55850674160, -52695000560, -46383566960, -43227893360, -40072219760, -33760786160, -30605112560, -27449438960, -21138005360, -17982331760, -14826658160, -12219278960, -2208988800 ], + "diffs" : [ -172240, -85840, 560, 86960, 173360, 259760, 346160, 432560, 518960, 605360, 691760, 778160, 864560, 560, 0 ] +}, { + "tz" : "America/Atikokan", + "switches" : [ -62135574812, -59006339612, -55850666012, -52694992412, -46383558812, -43227885212, -40072211612, -33760778012, -30605104412, -27449430812, -21137997212, -17982323612, -14826650012, -12219270812, -2366733212, -2208988800 ], + "diffs" : [ -176788, -90388, -3988, 82412, 168812, 255212, 341612, 428012, 514412, 600812, 687212, 773612, 860012, -3988, -3600, 0 ] +}, { + "tz" : "America/Atka", + "switches" : [ -62135640802, -59006405602, -55850732002, -52695058402, -46383624802, -43227951202, -40072277602, -33760844002, -30605170402, -27449496802, -21138063202, -17982389602, -14826716002, -12219336802, -3225223727, -2208988800 ], + "diffs" : [ -92798, -6398, 80002, 166402, 252802, 339202, 425602, 512002, 598402, 684802, 771202, 857602, 944002, 80002, -6398, 0 ] +}, { + "tz" : "America/Bahia", + "switches" : [ -62135587556, -59006352356, -55850678756, -52695005156, -46383571556, -43227897956, -40072224356, -33760790756, -30605117156, -27449443556, -21138009956, -17982336356, -14826662756, -12219283556, -2208988800 ], + "diffs" : [ -171244, -84844, 1556, 87956, 174356, 260756, 347156, 433556, 519956, 606356, 692756, 779156, 865556, 1556, 0 ] +}, { + "tz" : "America/Bahia_Banderas", + "switches" : [ -62135571540, -59006336340, -55850662740, -52694989140, -46383555540, -43227881940, -40072208340, -33760774740, -30605101140, -27449427540, -21137993940, -17982320340, -14826646740, -12219267540, -2208988800 ], + "diffs" : [ -176460, -90060, -3660, 82740, 169140, 255540, 341940, 428340, 514740, 601140, 687540, 773940, 860340, -3660, 0 ] +}, { + "tz" : "America/Barbados", + "switches" : [ -62135582491, -59006347291, -55850673691, -52695000091, -46383566491, -43227892891, -40072219291, -33760785691, -30605112091, -27449438491, -21138004891, -17982331291, -14826657691, -12219278491, -2208988800 ], + "diffs" : [ -172709, -86309, 91, 86491, 172891, 259291, 345691, 432091, 518491, 604891, 691291, 777691, 864091, 91, 0 ] +}, { + "tz" : "America/Belem", + "switches" : [ -62135585164, -59006349964, -55850676364, -52695002764, -46383569164, -43227895564, -40072221964, -33760788364, -30605114764, -27449441164, -21138007564, -17982333964, -14826660364, -12219281164, -2208988800 ], + "diffs" : [ -173636, -87236, -836, 85564, 171964, 258364, 344764, 431164, 517564, 603964, 690364, 776764, 863164, -836, 0 ] +}, { + "tz" : "America/Belize", + "switches" : [ -62135575632, -59006340432, -55850666832, -52694993232, -46383559632, -43227886032, -40072212432, -33760778832, -30605105232, -27449431632, -21137998032, -17982324432, -14826650832, -12219271632, -2208988800 ], + "diffs" : [ -172368, -85968, 432, 86832, 173232, 259632, 346032, 432432, 518832, 605232, 691632, 778032, 864432, 432, 0 ] +}, { + "tz" : "America/Blanc-Sablon", + "switches" : [ -62135583092, -59006347892, -55850674292, -52695000692, -46383567092, -43227893492, -40072219892, -33760786292, -30605112692, -27449439092, -21138005492, -17982331892, -14826658292, -12219279092, -2713896692 ], + "diffs" : [ -172108, -85708, 692, 87092, 173492, 259892, 346292, 432692, 519092, 605492, 691892, 778292, 864692, 692, 0 ] +}, { + "tz" : "America/Boa_Vista", + "switches" : [ -62135582240, -59006347040, -55850673440, -52694999840, -46383566240, -43227892640, -40072219040, -33760785440, -30605111840, -27449438240, -21138004640, -17982331040, -14826657440, -12219278240, -2208988800 ], + "diffs" : [ -172960, -86560, -160, 86240, 172640, 259040, 345440, 431840, 518240, 604640, 691040, 777440, 863840, -160, 0 ] +}, { + "tz" : "America/Bogota", + "switches" : [ -62135579024, -59006343824, -55850670224, -52694996624, -46383563024, -43227889424, -40072215824, -33760782224, -30605108624, -27449435024, -21138001424, -17982327824, -14826654224, -12219275024, -2208988800 ], + "diffs" : [ -172576, -86176, 224, 86624, 173024, 259424, 345824, 432224, 518624, 605024, 691424, 777824, 864224, 224, 0 ] +}, { + "tz" : "America/Boise", + "switches" : [ -62135568911, -59006333711, -55850660111, -52694986511, -46383552911, -43227879311, -40072205711, -33760772111, -30605098511, -27449424911, -21137991311, -17982317711, -14826644111, -12219264911, -2717640000, -2208988800 ], + "diffs" : [ -175489, -89089, -2689, 83711, 170111, 256511, 342911, 429311, 515711, 602111, 688511, 774911, 861311, -2689, -3600, 0 ] +}, { + "tz" : "America/Buenos_Aires", + "switches" : [ -62135582772, -59006347572, -55850673972, -52695000372, -46383566772, -43227893172, -40072219572, -33760785972, -30605112372, -27449438772, -21138005172, -17982331572, -14826657972, -12219278772, -2372097972, -2208988800 ], + "diffs" : [ -176028, -89628, -3228, 83172, 169572, 255972, 342372, 428772, 515172, 601572, 687972, 774372, 860772, -3228, -4608, 0 ] +}, { + "tz" : "America/Cambridge_Bay", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -147600, -61200, 25200, 111600, 198000, 284400, 370800, 457200, 543600, 630000, 716400, 802800, 889200, 25200, 0 ] +}, { + "tz" : "America/Campo_Grande", + "switches" : [ -62135583692, -59006348492, -55850674892, -52695001292, -46383567692, -43227894092, -40072220492, -33760786892, -30605113292, -27449439692, -21138006092, -17982332492, -14826658892, -12219279692, -2208988800 ], + "diffs" : [ -171508, -85108, 1292, 87692, 174092, 260492, 346892, 433292, 519692, 606092, 692492, 778892, 865292, 1292, 0 ] +}, { + "tz" : "America/Cancun", + "switches" : [ -62135575976, -59006340776, -55850667176, -52694993576, -46383559976, -43227886376, -40072212776, -33760779176, -30605105576, -27449431976, -21137998376, -17982324776, -14826651176, -12219271976, -2208988800 ], + "diffs" : [ -175624, -89224, -2824, 83576, 169976, 256376, 342776, 429176, 515576, 601976, 688376, 774776, 861176, -2824, 0 ] +}, { + "tz" : "America/Caracas", + "switches" : [ -62135580736, -59006345536, -55850671936, -52694998336, -46383564736, -43227891136, -40072217536, -33760783936, -30605110336, -27449436736, -21138003136, -17982329536, -14826655936, -12219276736, -2524505536, -2208988800 ], + "diffs" : [ -174464, -88064, -1664, 84736, 171136, 257536, 343936, 430336, 516736, 603136, 689536, 775936, 862336, -1664, -1660, 0 ] +}, { + "tz" : "America/Catamarca", + "switches" : [ -62135581012, -59006345812, -55850672212, -52694998612, -46383565012, -43227891412, -40072217812, -33760784212, -30605110612, -27449437012, -21138003412, -17982329812, -14826656212, -12219277012, -2372096212, -2208988800 ], + "diffs" : [ -177788, -91388, -4988, 81412, 167812, 254212, 340612, 427012, 513412, 599812, 686212, 772612, 859012, -4988, -4608, 0 ] +}, { + "tz" : "America/Cayenne", + "switches" : [ -62135584240, -59006349040, -55850675440, -52695001840, -46383568240, -43227894640, -40072221040, -33760787440, -30605113840, -27449440240, -21138006640, -17982333040, -14826659440, -12219280240, -2208988800 ], + "diffs" : [ -174560, -88160, -1760, 84640, 171040, 257440, 343840, 430240, 516640, 603040, 689440, 775840, 862240, -1760, 0 ] +}, { + "tz" : "America/Cayman", + "switches" : [ -62135577712, -59006342512, -55850668912, -52694995312, -46383561712, -43227888112, -40072214512, -33760780912, -30605107312, -27449433712, -21138000112, -17982326512, -14826652912, -12219273712, -2524502512, -2208988800 ], + "diffs" : [ -173888, -87488, -1088, 85312, 171712, 258112, 344512, 430912, 517312, 603712, 690112, 776512, 862912, -1088, -1176, 0 ] +}, { + "tz" : "America/Chicago", + "switches" : [ -62135575764, -59006340564, -55850666964, -52694993364, -46383559764, -43227886164, -40072212564, -33760778964, -30605105364, -27449431764, -21137998164, -17982324564, -14826650964, -12219271764, -2717647200 ], + "diffs" : [ -172236, -85836, 564, 86964, 173364, 259764, 346164, 432564, 518964, 605364, 691764, 778164, 864564, 564, 0 ] +}, { + "tz" : "America/Chihuahua", + "switches" : [ -62135571340, -59006336140, -55850662540, -52694988940, -46383555340, -43227881740, -40072208140, -33760774540, -30605100940, -27449427340, -21137993740, -17982320140, -14826646540, -12219267340, -2208988800 ], + "diffs" : [ -173060, -86660, -260, 86140, 172540, 258940, 345340, 431740, 518140, 604540, 690940, 777340, 863740, -260, 0 ] +}, { + "tz" : "America/Coral_Harbour", + "switches" : [ -62135574812, -59006339612, -55850666012, -52694992412, -46383558812, -43227885212, -40072211612, -33760778012, -30605104412, -27449430812, -21137997212, -17982323612, -14826650012, -12219270812, -2366733212, -2208988800 ], + "diffs" : [ -176788, -90388, -3988, 82412, 168812, 255212, 341612, 428012, 514412, 600812, 687212, 773612, 860012, -3988, -3600, 0 ] +}, { + "tz" : "America/Cordoba", + "switches" : [ -62135581392, -59006346192, -55850672592, -52694998992, -46383565392, -43227891792, -40072218192, -33760784592, -30605110992, -27449437392, -21138003792, -17982330192, -14826656592, -12219277392, -2208988800 ], + "diffs" : [ -177408, -91008, -4608, 81792, 168192, 254592, 340992, 427392, 513792, 600192, 686592, 772992, 859392, -4608, 0 ] +}, { + "tz" : "America/Costa_Rica", + "switches" : [ -62135576627, -59006341427, -55850667827, -52694994227, -46383560627, -43227887027, -40072213427, -33760779827, -30605106227, -27449432627, -21137999027, -17982325427, -14826651827, -12219272627, -2208988800 ], + "diffs" : [ -171373, -84973, 1427, 87827, 174227, 260627, 347027, 433427, 519827, 606227, 692627, 779027, 865427, 1427, 0 ] +}, { + "tz" : "America/Creston", + "switches" : [ -62135568836, -59006333636, -55850660036, -52694986436, -46383552836, -43227879236, -40072205636, -33760772036, -30605098436, -27449424836, -21137991236, -17982317636, -14826644036, -12219264836, -2713882436 ], + "diffs" : [ -175564, -89164, -2764, 83636, 170036, 256436, 342836, 429236, 515636, 602036, 688436, 774836, 861236, -2764, 0 ] +}, { + "tz" : "America/Cuiaba", + "switches" : [ -62135583340, -59006348140, -55850674540, -52695000940, -46383567340, -43227893740, -40072220140, -33760786540, -30605112940, -27449439340, -21138005740, -17982332140, -14826658540, -12219279340, -2208988800 ], + "diffs" : [ -171860, -85460, 940, 87340, 173740, 260140, 346540, 432940, 519340, 605740, 692140, 778540, 864940, 940, 0 ] +}, { + "tz" : "America/Curacao", + "switches" : [ -62135580253, -59006345053, -55850671453, -52694997853, -46383564253, -43227890653, -40072217053, -33760783453, -30605109853, -27449436253, -21138002653, -17982329053, -14826655453, -12219276253, -2208988800 ], + "diffs" : [ -174947, -88547, -2147, 84253, 170653, 257053, 343453, 429853, 516253, 602653, 689053, 775453, 861853, -2147, 0 ] +}, { + "tz" : "America/Danmarkshavn", + "switches" : [ -62135592320, -59006357120, -55850683520, -52695009920, -46383576320, -43227902720, -40072229120, -33760795520, -30605121920, -27449448320, -21138014720, -17982341120, -14826667520, -12219288320, -2208988800 ], + "diffs" : [ -177280, -90880, -4480, 81920, 168320, 254720, 341120, 427520, 513920, 600320, 686720, 773120, 859520, -4480, 0 ] +}, { + "tz" : "America/Dawson", + "switches" : [ -62135563340, -59006328140, -55850654540, -52694980940, -46383547340, -43227873740, -40072200140, -33760766540, -30605092940, -27449419340, -21137985740, -17982312140, -14826638540, -12219259340, -2208988800 ], + "diffs" : [ -177460, -91060, -4660, 81740, 168140, 254540, 340940, 427340, 513740, 600140, 686540, 772940, 859340, -4660, 0 ] +}, { + "tz" : "America/Dawson_Creek", + "switches" : [ -62135567944, -59006332744, -55850659144, -52694985544, -46383551944, -43227878344, -40072204744, -33760771144, -30605097544, -27449423944, -21137990344, -17982316744, -14826643144, -12219263944, -2713881544, -2208988800 ], + "diffs" : [ -176456, -90056, -3656, 82744, 169144, 255544, 341944, 428344, 514744, 601144, 687544, 773944, 860344, -3656, -3600, 0 ] +}, { + "tz" : "America/Denver", + "switches" : [ -62135571604, -59006336404, -55850662804, -52694989204, -46383555604, -43227882004, -40072208404, -33760774804, -30605101204, -27449427604, -21137994004, -17982320404, -14826646804, -12219267604, -2717643600 ], + "diffs" : [ -172796, -86396, 4, 86404, 172804, 259204, 345604, 432004, 518404, 604804, 691204, 777604, 864004, 4, 0 ] +}, { + "tz" : "America/Detroit", + "switches" : [ -62135576869, -59006341669, -55850668069, -52694994469, -46383560869, -43227887269, -40072213669, -33760780069, -30605106469, -27449432869, -21137999269, -17982325669, -14826652069, -12219272869, -2208988800 ], + "diffs" : [ -174731, -88331, -1931, 84469, 170869, 257269, 343669, 430069, 516469, 602869, 689269, 775669, 862069, -1931, 0 ] +}, { + "tz" : "America/Dominica", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Edmonton", + "switches" : [ -62135569568, -59006334368, -55850660768, -52694987168, -46383553568, -43227879968, -40072206368, -33760772768, -30605099168, -27449425568, -21137991968, -17982318368, -14826644768, -12219265568, -2208988800 ], + "diffs" : [ -174832, -88432, -2032, 84368, 170768, 257168, 343568, 429968, 516368, 602768, 689168, 775568, 861968, -2032, 0 ] +}, { + "tz" : "America/Eirunepe", + "switches" : [ -62135580032, -59006344832, -55850671232, -52694997632, -46383564032, -43227890432, -40072216832, -33760783232, -30605109632, -27449436032, -21138002432, -17982328832, -14826655232, -12219276032, -2208988800 ], + "diffs" : [ -171568, -85168, 1232, 87632, 174032, 260432, 346832, 433232, 519632, 606032, 692432, 778832, 865232, 1232, 0 ] +}, { + "tz" : "America/El_Salvador", + "switches" : [ -62135575392, -59006340192, -55850666592, -52694992992, -46383559392, -43227885792, -40072212192, -33760778592, -30605104992, -27449431392, -21137997792, -17982324192, -14826650592, -12219271392, -2208988800 ], + "diffs" : [ -172608, -86208, 192, 86592, 172992, 259392, 345792, 432192, 518592, 604992, 691392, 777792, 864192, 192, 0 ] +}, { + "tz" : "America/Ensenada", + "switches" : [ -62135568716, -59006333516, -55850659916, -52694986316, -46383552716, -43227879116, -40072205516, -33760771916, -30605098316, -27449424716, -21137991116, -17982317516, -14826643916, -12219264716, -2208988800 ], + "diffs" : [ -172084, -85684, 716, 87116, 173516, 259916, 346316, 432716, 519116, 605516, 691916, 778316, 864716, 716, 0 ] +}, { + "tz" : "America/Fort_Nelson", + "switches" : [ -62135567353, -59006332153, -55850658553, -52694984953, -46383551353, -43227877753, -40072204153, -33760770553, -30605096953, -27449423353, -21137989753, -17982316153, -14826642553, -12219263353, -2713880953, -2208988800 ], + "diffs" : [ -177047, -90647, -4247, 82153, 168553, 254953, 341353, 427753, 514153, 600553, 686953, 773353, 859753, -4247, -3600, 0 ] +}, { + "tz" : "America/Fort_Wayne", + "switches" : [ -62135576122, -59006340922, -55850667322, -52694993722, -46383560122, -43227886522, -40072212922, -33760779322, -30605105722, -27449432122, -21137998522, -17982324922, -14826651322, -12219272122, -2717647200, -2208988800 ], + "diffs" : [ -175478, -89078, -2678, 83722, 170122, 256522, 342922, 429322, 515722, 602122, 688522, 774922, 861322, -2678, -3600, 0 ] +}, { + "tz" : "America/Fortaleza", + "switches" : [ -62135587560, -59006352360, -55850678760, -52695005160, -46383571560, -43227897960, -40072224360, -33760790760, -30605117160, -27449443560, -21138009960, -17982336360, -14826662760, -12219283560, -2208988800 ], + "diffs" : [ -171240, -84840, 1560, 87960, 174360, 260760, 347160, 433560, 519960, 606360, 692760, 779160, 865560, 1560, 0 ] +}, { + "tz" : "America/Glace_Bay", + "switches" : [ -62135582412, -59006347212, -55850673612, -52695000012, -46383566412, -43227892812, -40072219212, -33760785612, -30605112012, -27449438412, -21138004812, -17982331212, -14826657612, -12219278412, -2208988800 ], + "diffs" : [ -172788, -86388, 12, 86412, 172812, 259212, 345612, 432012, 518412, 604812, 691212, 777612, 864012, 12, 0 ] +}, { + "tz" : "America/Godthab", + "switches" : [ -62135584384, -59006349184, -55850675584, -52695001984, -46383568384, -43227894784, -40072221184, -33760787584, -30605113984, -27449440384, -21138006784, -17982333184, -14826659584, -12219280384, -2208988800 ], + "diffs" : [ -174416, -88016, -1616, 84784, 171184, 257584, 343984, 430384, 516784, 603184, 689584, 775984, 862384, -1616, 0 ] +}, { + "tz" : "America/Goose_Bay", + "switches" : [ -62135582300, -59006347100, -55850673500, -52694999900, -46383566300, -43227892700, -40072219100, -33760785500, -30605111900, -27449438300, -21138004700, -17982331100, -14826657500, -12219278300, -2713895900, -2208988800 ], + "diffs" : [ -172900, -86500, -100, 86300, 172700, 259100, 345500, 431900, 518300, 604700, 691100, 777500, 863900, -100, 1748, 0 ] +}, { + "tz" : "America/Grand_Turk", + "switches" : [ -62135579728, -59006344528, -55850670928, -52694997328, -46383563728, -43227890128, -40072216528, -33760782928, -30605109328, -27449435728, -21138002128, -17982328528, -14826654928, -12219275728, -2524504528, -2208988800 ], + "diffs" : [ -171872, -85472, 928, 87328, 173728, 260128, 346528, 432928, 519328, 605728, 692128, 778528, 864928, 928, -430, 0 ] +}, { + "tz" : "America/Grenada", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Guadeloupe", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Guatemala", + "switches" : [ -62135575076, -59006339876, -55850666276, -52694992676, -46383559076, -43227885476, -40072211876, -33760778276, -30605104676, -27449431076, -21137997476, -17982323876, -14826650276, -12219271076, -2208988800 ], + "diffs" : [ -172924, -86524, -124, 86276, 172676, 259076, 345476, 431876, 518276, 604676, 691076, 777476, 863876, -124, 0 ] +}, { + "tz" : "America/Guayaquil", + "switches" : [ -62135577640, -59006342440, -55850668840, -52694995240, -46383561640, -43227888040, -40072214440, -33760780840, -30605107240, -27449433640, -21138000040, -17982326440, -14826652840, -12219273640, -2524502440, -2208988800 ], + "diffs" : [ -173960, -87560, -1160, 85240, 171640, 258040, 344440, 430840, 517240, 603640, 690040, 776440, 862840, -1160, -840, 0 ] +}, { + "tz" : "America/Guyana", + "switches" : [ -62135582840, -59006347640, -55850674040, -52695000440, -46383566840, -43227893240, -40072219640, -33760786040, -30605112440, -27449438840, -21138005240, -17982331640, -14826658040, -12219278840, -2208988800 ], + "diffs" : [ -172360, -85960, 440, 86840, 173240, 259640, 346040, 432440, 518840, 605240, 691640, 778040, 864440, 440, 0 ] +}, { + "tz" : "America/Halifax", + "switches" : [ -62135581536, -59006346336, -55850672736, -52694999136, -46383565536, -43227891936, -40072218336, -33760784736, -30605111136, -27449437536, -21138003936, -17982330336, -14826656736, -12219277536, -2208988800 ], + "diffs" : [ -173664, -87264, -864, 85536, 171936, 258336, 344736, 431136, 517536, 603936, 690336, 776736, 863136, -864, 0 ] +}, { + "tz" : "America/Havana", + "switches" : [ -62135577032, -59006341832, -55850668232, -52694994632, -46383561032, -43227887432, -40072213832, -33760780232, -30605106632, -27449433032, -21137999432, -17982325832, -14826652232, -12219273032, -2524501832, -2208988800 ], + "diffs" : [ -174568, -88168, -1768, 84632, 171032, 257432, 343832, 430232, 516632, 603032, 689432, 775832, 862232, -1768, -1776, 0 ] +}, { + "tz" : "America/Hermosillo", + "switches" : [ -62135570168, -59006334968, -55850661368, -52694987768, -46383554168, -43227880568, -40072206968, -33760773368, -30605099768, -27449426168, -21137992568, -17982318968, -14826645368, -12219266168, -2208988800 ], + "diffs" : [ -174232, -87832, -1432, 84968, 171368, 257768, 344168, 430568, 516968, 603368, 689768, 776168, 862568, -1432, 0 ] +}, { + "tz" : "America/Indiana/Indianapolis", + "switches" : [ -62135576122, -59006340922, -55850667322, -52694993722, -46383560122, -43227886522, -40072212922, -33760779322, -30605105722, -27449432122, -21137998522, -17982324922, -14826651322, -12219272122, -2717647200, -2208988800 ], + "diffs" : [ -175478, -89078, -2678, 83722, 170122, 256522, 342922, 429322, 515722, 602122, 688522, 774922, 861322, -2678, -3600, 0 ] +}, { + "tz" : "America/Indiana/Knox", + "switches" : [ -62135576010, -59006340810, -55850667210, -52694993610, -46383560010, -43227886410, -40072212810, -33760779210, -30605105610, -27449432010, -21137998410, -17982324810, -14826651210, -12219272010, -2717647200 ], + "diffs" : [ -171990, -85590, 810, 87210, 173610, 260010, 346410, 432810, 519210, 605610, 692010, 778410, 864810, 810, 0 ] +}, { + "tz" : "America/Indiana/Marengo", + "switches" : [ -62135576077, -59006340877, -55850667277, -52694993677, -46383560077, -43227886477, -40072212877, -33760779277, -30605105677, -27449432077, -21137998477, -17982324877, -14826651277, -12219272077, -2717647200, -2208988800 ], + "diffs" : [ -175523, -89123, -2723, 83677, 170077, 256477, 342877, 429277, 515677, 602077, 688477, 774877, 861277, -2723, -3600, 0 ] +}, { + "tz" : "America/Indiana/Petersburg", + "switches" : [ -62135575853, -59006340653, -55850667053, -52694993453, -46383559853, -43227886253, -40072212653, -33760779053, -30605105453, -27449431853, -21137998253, -17982324653, -14826651053, -12219271853, -2717647200, -2208988800 ], + "diffs" : [ -175747, -89347, -2947, 83453, 169853, 256253, 342653, 429053, 515453, 601853, 688253, 774653, 861053, -2947, -3600, 0 ] +}, { + "tz" : "America/Indiana/Tell_City", + "switches" : [ -62135575977, -59006340777, -55850667177, -52694993577, -46383559977, -43227886377, -40072212777, -33760779177, -30605105577, -27449431977, -21137998377, -17982324777, -14826651177, -12219271977, -2717647200 ], + "diffs" : [ -172023, -85623, 777, 87177, 173577, 259977, 346377, 432777, 519177, 605577, 691977, 778377, 864777, 777, 0 ] +}, { + "tz" : "America/Indiana/Vevay", + "switches" : [ -62135576384, -59006341184, -55850667584, -52694993984, -46383560384, -43227886784, -40072213184, -33760779584, -30605105984, -27449432384, -21137998784, -17982325184, -14826651584, -12219272384, -2717647200, -2208988800 ], + "diffs" : [ -175216, -88816, -2416, 83984, 170384, 256784, 343184, 429584, 515984, 602384, 688784, 775184, 861584, -2416, -3600, 0 ] +}, { + "tz" : "America/Indiana/Vincennes", + "switches" : [ -62135575793, -59006340593, -55850666993, -52694993393, -46383559793, -43227886193, -40072212593, -33760778993, -30605105393, -27449431793, -21137998193, -17982324593, -14826650993, -12219271793, -2717647200, -2208988800 ], + "diffs" : [ -175807, -89407, -3007, 83393, 169793, 256193, 342593, 428993, 515393, 601793, 688193, 774593, 860993, -3007, -3600, 0 ] +}, { + "tz" : "America/Indiana/Winamac", + "switches" : [ -62135576015, -59006340815, -55850667215, -52694993615, -46383560015, -43227886415, -40072212815, -33760779215, -30605105615, -27449432015, -21137998415, -17982324815, -14826651215, -12219272015, -2717647200, -2208988800 ], + "diffs" : [ -175585, -89185, -2785, 83615, 170015, 256415, 342815, 429215, 515615, 602015, 688415, 774815, 861215, -2785, -3600, 0 ] +}, { + "tz" : "America/Indianapolis", + "switches" : [ -62135576122, -59006340922, -55850667322, -52694993722, -46383560122, -43227886522, -40072212922, -33760779322, -30605105722, -27449432122, -21137998522, -17982324922, -14826651322, -12219272122, -2717647200, -2208988800 ], + "diffs" : [ -175478, -89078, -2678, 83722, 170122, 256522, 342922, 429322, 515722, 602122, 688522, 774922, 861322, -2678, -3600, 0 ] +}, { + "tz" : "America/Inuvik", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -147600, -61200, 25200, 111600, 198000, 284400, 370800, 457200, 543600, 630000, 716400, 802800, 889200, 25200, 0 ] +}, { + "tz" : "America/Iqaluit", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -154800, -68400, 18000, 104400, 190800, 277200, 363600, 450000, 536400, 622800, 709200, 795600, 882000, 18000, 0 ] +}, { + "tz" : "America/Jamaica", + "switches" : [ -62135578370, -59006343170, -55850669570, -52694995970, -46383562370, -43227888770, -40072215170, -33760781570, -30605107970, -27449434370, -21138000770, -17982327170, -14826653570, -12219274370, -2208988800 ], + "diffs" : [ -173230, -86830, -430, 85970, 172370, 258770, 345170, 431570, 517970, 604370, 690770, 777170, 863570, -430, 0 ] +}, { + "tz" : "America/Jujuy", + "switches" : [ -62135581128, -59006345928, -55850672328, -52694998728, -46383565128, -43227891528, -40072217928, -33760784328, -30605110728, -27449437128, -21138003528, -17982329928, -14826656328, -12219277128, -2372096328, -2208988800 ], + "diffs" : [ -177672, -91272, -4872, 81528, 167928, 254328, 340728, 427128, 513528, 599928, 686328, 772728, 859128, -4872, -4608, 0 ] +}, { + "tz" : "America/Juneau", + "switches" : [ -62135650939, -59006415739, -55850742139, -52695068539, -46383634939, -43227961339, -40072287739, -33760854139, -30605180539, -27449506939, -21138073339, -17982399739, -14826726139, -12219346939, -3225223727, -2208988800 ], + "diffs" : [ -86261, 139, 86539, 172939, 259339, 345739, 432139, 518539, 604939, 691339, 777739, 864139, 950539, 86539, 139, 0 ] +}, { + "tz" : "America/Kentucky/Louisville", + "switches" : [ -62135576218, -59006341018, -55850667418, -52694993818, -46383560218, -43227886618, -40072213018, -33760779418, -30605105818, -27449432218, -21137998618, -17982325018, -14826651418, -12219272218, -2717647200, -2208988800 ], + "diffs" : [ -175382, -88982, -2582, 83818, 170218, 256618, 343018, 429418, 515818, 602218, 688618, 775018, 861418, -2582, -3600, 0 ] +}, { + "tz" : "America/Kentucky/Monticello", + "switches" : [ -62135576436, -59006341236, -55850667636, -52694994036, -46383560436, -43227886836, -40072213236, -33760779636, -30605106036, -27449432436, -21137998836, -17982325236, -14826651636, -12219272436, -2717647200, -2208988800 ], + "diffs" : [ -175164, -88764, -2364, 84036, 170436, 256836, 343236, 429636, 516036, 602436, 688836, 775236, 861636, -2364, -3600, 0 ] +}, { + "tz" : "America/Knox_IN", + "switches" : [ -62135576010, -59006340810, -55850667210, -52694993610, -46383560010, -43227886410, -40072212810, -33760779210, -30605105610, -27449432010, -21137998410, -17982324810, -14826651210, -12219272010, -2717647200 ], + "diffs" : [ -171990, -85590, 810, 87210, 173610, 260010, 346410, 432810, 519210, 605610, 692010, 778410, 864810, 810, 0 ] +}, { + "tz" : "America/Kralendijk", + "switches" : [ -62135580253, -59006345053, -55850671453, -52694997853, -46383564253, -43227890653, -40072217053, -33760783453, -30605109853, -27449436253, -21138002653, -17982329053, -14826655453, -12219276253, -2208988800 ], + "diffs" : [ -174947, -88547, -2147, 84253, 170653, 257053, 343453, 429853, 516253, 602653, 689053, 775453, 861853, -2147, 0 ] +}, { + "tz" : "America/La_Paz", + "switches" : [ -62135580444, -59006345244, -55850671644, -52694998044, -46383564444, -43227890844, -40072217244, -33760783644, -30605110044, -27449436444, -21138002844, -17982329244, -14826655644, -12219276444, -2208988800 ], + "diffs" : [ -174756, -88356, -1956, 84444, 170844, 257244, 343644, 430044, 516444, 602844, 689244, 775644, 862044, -1956, 0 ] +}, { + "tz" : "America/Lima", + "switches" : [ -62135578308, -59006343108, -55850669508, -52694995908, -46383562308, -43227888708, -40072215108, -33760781508, -30605107908, -27449434308, -21138000708, -17982327108, -14826653508, -12219274308, -2524503108, -2208988800 ], + "diffs" : [ -173292, -86892, -492, 85908, 172308, 258708, 345108, 431508, 517908, 604308, 690708, 777108, 863508, -492, -516, 0 ] +}, { + "tz" : "America/Los_Angeles", + "switches" : [ -62135568422, -59006333222, -55850659622, -52694986022, -46383552422, -43227878822, -40072205222, -33760771622, -30605098022, -27449424422, -21137990822, -17982317222, -14826643622, -12219264422, -2717640000 ], + "diffs" : [ -172378, -85978, 422, 86822, 173222, 259622, 346022, 432422, 518822, 605222, 691622, 778022, 864422, 422, 0 ] +}, { + "tz" : "America/Louisville", + "switches" : [ -62135576218, -59006341018, -55850667418, -52694993818, -46383560218, -43227886618, -40072213018, -33760779418, -30605105818, -27449432218, -21137998618, -17982325018, -14826651418, -12219272218, -2717647200, -2208988800 ], + "diffs" : [ -175382, -88982, -2582, 83818, 170218, 256618, 343018, 429418, 515818, 602218, 688618, 775018, 861418, -2582, -3600, 0 ] +}, { + "tz" : "America/Lower_Princes", + "switches" : [ -62135580253, -59006345053, -55850671453, -52694997853, -46383564253, -43227890653, -40072217053, -33760783453, -30605109853, -27449436253, -21138002653, -17982329053, -14826655453, -12219276253, -2208988800 ], + "diffs" : [ -174947, -88547, -2147, 84253, 170653, 257053, 343453, 429853, 516253, 602653, 689053, 775453, 861853, -2147, 0 ] +}, { + "tz" : "America/Maceio", + "switches" : [ -62135588228, -59006353028, -55850679428, -52695005828, -46383572228, -43227898628, -40072225028, -33760791428, -30605117828, -27449444228, -21138010628, -17982337028, -14826663428, -12219284228, -2208988800 ], + "diffs" : [ -170572, -84172, 2228, 88628, 175028, 261428, 347828, 434228, 520628, 607028, 693428, 779828, 866228, 2228, 0 ] +}, { + "tz" : "America/Managua", + "switches" : [ -62135576092, -59006340892, -55850667292, -52694993692, -46383560092, -43227886492, -40072212892, -33760779292, -30605105692, -27449432092, -21137998492, -17982324892, -14826651292, -12219272092, -2524500892, -2208988800 ], + "diffs" : [ -171908, -85508, 892, 87292, 173692, 260092, 346492, 432892, 519292, 605692, 692092, 778492, 864892, 892, 888, 0 ] +}, { + "tz" : "America/Manaus", + "switches" : [ -62135582396, -59006347196, -55850673596, -52694999996, -46383566396, -43227892796, -40072219196, -33760785596, -30605111996, -27449438396, -21138004796, -17982331196, -14826657596, -12219278396, -2208988800 ], + "diffs" : [ -172804, -86404, -4, 86396, 172796, 259196, 345596, 431996, 518396, 604796, 691196, 777596, 863996, -4, 0 ] +}, { + "tz" : "America/Marigot", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Martinique", + "switches" : [ -62135582140, -59006346940, -55850673340, -52694999740, -46383566140, -43227892540, -40072218940, -33760785340, -30605111740, -27449438140, -21138004540, -17982330940, -14826657340, -12219278140, -2208988800 ], + "diffs" : [ -173060, -86660, -260, 86140, 172540, 258940, 345340, 431740, 518140, 604540, 690940, 777340, 863740, -260, 0 ] +}, { + "tz" : "America/Matamoros", + "switches" : [ -62135572800, -59006337600, -55850664000, -52694990400, -46383556800, -43227883200, -40072209600, -33760776000, -30605102400, -27449428800, -21137995200, -17982321600, -14826648000, -12219268800, -2208988800 ], + "diffs" : [ -175200, -88800, -2400, 84000, 170400, 256800, 343200, 429600, 516000, 602400, 688800, 775200, 861600, -2400, 0 ] +}, { + "tz" : "America/Mazatlan", + "switches" : [ -62135571260, -59006336060, -55850662460, -52694988860, -46383555260, -43227881660, -40072208060, -33760774460, -30605100860, -27449427260, -21137993660, -17982320060, -14826646460, -12219267260, -2208988800 ], + "diffs" : [ -173140, -86740, -340, 86060, 172460, 258860, 345260, 431660, 518060, 604460, 690860, 777260, 863660, -340, 0 ] +}, { + "tz" : "America/Mendoza", + "switches" : [ -62135580284, -59006345084, -55850671484, -52694997884, -46383564284, -43227890684, -40072217084, -33760783484, -30605109884, -27449436284, -21138002684, -17982329084, -14826655484, -12219276284, -2372095484, -2208988800 ], + "diffs" : [ -178516, -92116, -5716, 80684, 167084, 253484, 339884, 426284, 512684, 599084, 685484, 771884, 858284, -5716, -4608, 0 ] +}, { + "tz" : "America/Menominee", + "switches" : [ -62135575773, -59006340573, -55850666973, -52694993373, -46383559773, -43227886173, -40072212573, -33760778973, -30605105373, -27449431773, -21137998173, -17982324573, -14826650973, -12219271773, -2659759773 ], + "diffs" : [ -172227, -85827, 573, 86973, 173373, 259773, 346173, 432573, 518973, 605373, 691773, 778173, 864573, 573, 0 ] +}, { + "tz" : "America/Merida", + "switches" : [ -62135575292, -59006340092, -55850666492, -52694992892, -46383559292, -43227885692, -40072212092, -33760778492, -30605104892, -27449431292, -21137997692, -17982324092, -14826650492, -12219271292, -2208988800 ], + "diffs" : [ -172708, -86308, 92, 86492, 172892, 259292, 345692, 432092, 518492, 604892, 691292, 777692, 864092, 92, 0 ] +}, { + "tz" : "America/Metlakatla", + "switches" : [ -62135651622, -59006416422, -55850742822, -52695069222, -46383635622, -43227962022, -40072288422, -33760854822, -30605181222, -27449507622, -21138074022, -17982400422, -14826726822, -12219347622, -3225223727, -2208988800 ], + "diffs" : [ -85578, 822, 87222, 173622, 260022, 346422, 432822, 519222, 605622, 692022, 778422, 864822, 951222, 87222, 822, 0 ] +}, { + "tz" : "America/Mexico_City", + "switches" : [ -62135573004, -59006337804, -55850664204, -52694990604, -46383557004, -43227883404, -40072209804, -33760776204, -30605102604, -27449429004, -21137995404, -17982321804, -14826648204, -12219269004, -2208988800 ], + "diffs" : [ -174996, -88596, -2196, 84204, 170604, 257004, 343404, 429804, 516204, 602604, 689004, 775404, 861804, -2196, 0 ] +}, { + "tz" : "America/Miquelon", + "switches" : [ -62135583320, -59006348120, -55850674520, -52695000920, -46383567320, -43227893720, -40072220120, -33760786520, -30605112920, -27449439320, -21138005720, -17982332120, -14826658520, -12219279320, -2208988800 ], + "diffs" : [ -175480, -89080, -2680, 83720, 170120, 256520, 342920, 429320, 515720, 602120, 688520, 774920, 861320, -2680, 0 ] +}, { + "tz" : "America/Moncton", + "switches" : [ -62135581252, -59006346052, -55850672452, -52694998852, -46383565252, -43227891652, -40072218052, -33760784452, -30605110852, -27449437252, -21138003652, -17982330052, -14826656452, -12219277252, -2715882052, -2208988800 ], + "diffs" : [ -173948, -87548, -1148, 85252, 171652, 258052, 344452, 430852, 517252, 603652, 690052, 776452, 862852, -1148, -3600, 0 ] +}, { + "tz" : "America/Monterrey", + "switches" : [ -62135572724, -59006337524, -55850663924, -52694990324, -46383556724, -43227883124, -40072209524, -33760775924, -30605102324, -27449428724, -21137995124, -17982321524, -14826647924, -12219268724, -2208988800 ], + "diffs" : [ -175276, -88876, -2476, 83924, 170324, 256724, 343124, 429524, 515924, 602324, 688724, 775124, 861524, -2476, 0 ] +}, { + "tz" : "America/Montevideo", + "switches" : [ -62135583309, -59006348109, -55850674509, -52695000909, -46383567309, -43227893709, -40072220109, -33760786509, -30605112909, -27449439309, -21138005709, -17982332109, -14826658509, -12219279309, -2208988800 ], + "diffs" : [ -175491, -89091, -2691, 83709, 170109, 256509, 342909, 429309, 515709, 602109, 688509, 774909, 861309, -2691, 0 ] +}, { + "tz" : "America/Montreal", + "switches" : [ -62135577748, -59006342548, -55850668948, -52694995348, -46383561748, -43227888148, -40072214548, -33760780948, -30605107348, -27449433748, -21138000148, -17982326548, -14826652948, -12219273748, -2366736148 ], + "diffs" : [ -173852, -87452, -1052, 85348, 171748, 258148, 344548, 430948, 517348, 603748, 690148, 776548, 862948, -1052, 0 ] +}, { + "tz" : "America/Montserrat", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Nassau", + "switches" : [ -62135578230, -59006343030, -55850669430, -52694995830, -46383562230, -43227888630, -40072215030, -33760781430, -30605107830, -27449434230, -21138000630, -17982327030, -14826653430, -12219274230, -2208988800 ], + "diffs" : [ -173370, -86970, -570, 85830, 172230, 258630, 345030, 431430, 517830, 604230, 690630, 777030, 863430, -570, 0 ] +}, { + "tz" : "America/New_York", + "switches" : [ -62135579038, -59006343838, -55850670238, -52694996638, -46383563038, -43227889438, -40072215838, -33760782238, -30605108638, -27449435038, -21138001438, -17982327838, -14826654238, -12219275038, -2717650800 ], + "diffs" : [ -172562, -86162, 238, 86638, 173038, 259438, 345838, 432238, 518638, 605038, 691438, 777838, 864238, 238, 0 ] +}, { + "tz" : "America/Nipigon", + "switches" : [ -62135575616, -59006340416, -55850666816, -52694993216, -46383559616, -43227886016, -40072212416, -33760778816, -30605105216, -27449431616, -21137998016, -17982324416, -14826650816, -12219271616, -2366734016 ], + "diffs" : [ -175984, -89584, -3184, 83216, 169616, 256016, 342416, 428816, 515216, 601616, 688016, 774416, 860816, -3184, 0 ] +}, { + "tz" : "America/Nome", + "switches" : [ -62135643502, -59006408302, -55850734702, -52695061102, -46383627502, -43227953902, -40072280302, -33760846702, -30605173102, -27449499502, -21138065902, -17982392302, -14826718702, -12219339502, -3225223727, -2208988800 ], + "diffs" : [ -93698, -7298, 79102, 165502, 251902, 338302, 424702, 511102, 597502, 683902, 770302, 856702, 943102, 79102, -7298, 0 ] +}, { + "tz" : "America/Noronha", + "switches" : [ -62135589020, -59006353820, -55850680220, -52695006620, -46383573020, -43227899420, -40072225820, -33760792220, -30605118620, -27449445020, -21138011420, -17982337820, -14826664220, -12219285020, -2208988800 ], + "diffs" : [ -173380, -86980, -580, 85820, 172220, 258620, 345020, 431420, 517820, 604220, 690620, 777020, 863420, -580, 0 ] +}, { + "tz" : "America/North_Dakota/Beulah", + "switches" : [ -62135572373, -59006337173, -55850663573, -52694989973, -46383556373, -43227882773, -40072209173, -33760775573, -30605101973, -27449428373, -21137994773, -17982321173, -14826647573, -12219268373, -2717643600, -2208988800 ], + "diffs" : [ -175627, -89227, -2827, 83573, 169973, 256373, 342773, 429173, 515573, 601973, 688373, 774773, 861173, -2827, -3600, 0 ] +}, { + "tz" : "America/North_Dakota/Center", + "switches" : [ -62135572488, -59006337288, -55850663688, -52694990088, -46383556488, -43227882888, -40072209288, -33760775688, -30605102088, -27449428488, -21137994888, -17982321288, -14826647688, -12219268488, -2717643600, -2208988800 ], + "diffs" : [ -175512, -89112, -2712, 83688, 170088, 256488, 342888, 429288, 515688, 602088, 688488, 774888, 861288, -2712, -3600, 0 ] +}, { + "tz" : "America/North_Dakota/New_Salem", + "switches" : [ -62135572461, -59006337261, -55850663661, -52694990061, -46383556461, -43227882861, -40072209261, -33760775661, -30605102061, -27449428461, -21137994861, -17982321261, -14826647661, -12219268461, -2717643600, -2208988800 ], + "diffs" : [ -175539, -89139, -2739, 83661, 170061, 256461, 342861, 429261, 515661, 602061, 688461, 774861, 861261, -2739, -3600, 0 ] +}, { + "tz" : "America/Ojinaga", + "switches" : [ -62135571740, -59006336540, -55850662940, -52694989340, -46383555740, -43227882140, -40072208540, -33760774940, -30605101340, -27449427740, -21137994140, -17982320540, -14826646940, -12219267740, -2208988800 ], + "diffs" : [ -172660, -86260, 140, 86540, 172940, 259340, 345740, 432140, 518540, 604940, 691340, 777740, 864140, 140, 0 ] +}, { + "tz" : "America/Panama", + "switches" : [ -62135577712, -59006342512, -55850668912, -52694995312, -46383561712, -43227888112, -40072214512, -33760780912, -30605107312, -27449433712, -21138000112, -17982326512, -14826652912, -12219273712, -2524502512, -2208988800 ], + "diffs" : [ -173888, -87488, -1088, 85312, 171712, 258112, 344512, 430912, 517312, 603712, 690112, 776512, 862912, -1088, -1176, 0 ] +}, { + "tz" : "America/Pangnirtung", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -154800, -68400, 18000, 104400, 190800, 277200, 363600, 450000, 536400, 622800, 709200, 795600, 882000, 18000, 0 ] +}, { + "tz" : "America/Paramaribo", + "switches" : [ -62135583560, -59006348360, -55850674760, -52695001160, -46383567560, -43227893960, -40072220360, -33760786760, -30605113160, -27449439560, -21138005960, -17982332360, -14826658760, -12219279560, -2208988800 ], + "diffs" : [ -175240, -88840, -2440, 83960, 170360, 256760, 343160, 429560, 515960, 602360, 688760, 775160, 861560, -2440, 0 ] +}, { + "tz" : "America/Phoenix", + "switches" : [ -62135569902, -59006334702, -55850661102, -52694987502, -46383553902, -43227880302, -40072206702, -33760773102, -30605099502, -27449425902, -21137992302, -17982318702, -14826645102, -12219265902, -2717643600 ], + "diffs" : [ -174498, -88098, -1698, 84702, 171102, 257502, 343902, 430302, 516702, 603102, 689502, 775902, 862302, -1698, 0 ] +}, { + "tz" : "America/Port-au-Prince", + "switches" : [ -62135579440, -59006344240, -55850670640, -52694997040, -46383563440, -43227889840, -40072216240, -33760782640, -30605109040, -27449435440, -21138001840, -17982328240, -14826654640, -12219275440, -2524504240, -2208988800 ], + "diffs" : [ -172160, -85760, 640, 87040, 173440, 259840, 346240, 432640, 519040, 605440, 691840, 778240, 864640, 640, 660, 0 ] +}, { + "tz" : "America/Port_of_Spain", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Porto_Acre", + "switches" : [ -62135580528, -59006345328, -55850671728, -52694998128, -46383564528, -43227890928, -40072217328, -33760783728, -30605110128, -27449436528, -21138002928, -17982329328, -14826655728, -12219276528, -2208988800 ], + "diffs" : [ -171072, -84672, 1728, 88128, 174528, 260928, 347328, 433728, 520128, 606528, 692928, 779328, 865728, 1728, 0 ] +}, { + "tz" : "America/Porto_Velho", + "switches" : [ -62135581464, -59006346264, -55850672664, -52694999064, -46383565464, -43227891864, -40072218264, -33760784664, -30605111064, -27449437464, -21138003864, -17982330264, -14826656664, -12219277464, -2208988800 ], + "diffs" : [ -173736, -87336, -936, 85464, 171864, 258264, 344664, 431064, 517464, 603864, 690264, 776664, 863064, -936, 0 ] +}, { + "tz" : "America/Puerto_Rico", + "switches" : [ -62135580935, -59006345735, -55850672135, -52694998535, -46383564935, -43227891335, -40072217735, -33760784135, -30605110535, -27449436935, -21138003335, -17982329735, -14826656135, -12219276935, -2233035335 ], + "diffs" : [ -174265, -87865, -1465, 84935, 171335, 257735, 344135, 430535, 516935, 603335, 689735, 776135, 862535, -1465, 0 ] +}, { + "tz" : "America/Punta_Arenas", + "switches" : [ -62135579780, -59006344580, -55850670980, -52694997380, -46383563780, -43227890180, -40072216580, -33760782980, -30605109380, -27449435780, -21138002180, -17982328580, -14826654980, -12219275780, -2524504580, -2208988800 ], + "diffs" : [ -179020, -92620, -6220, 80180, 166580, 252980, 339380, 425780, 512180, 598580, 684980, 771380, 857780, -6220, -6166, 0 ] +}, { + "tz" : "America/Rainy_River", + "switches" : [ -62135574104, -59006338904, -55850665304, -52694991704, -46383558104, -43227884504, -40072210904, -33760777304, -30605103704, -27449430104, -21137996504, -17982322904, -14826649304, -12219270104, -2366732504 ], + "diffs" : [ -173896, -87496, -1096, 85304, 171704, 258104, 344504, 430904, 517304, 603704, 690104, 776504, 862904, -1096, 0 ] +}, { + "tz" : "America/Rankin_Inlet", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -151200, -64800, 21600, 108000, 194400, 280800, 367200, 453600, 540000, 626400, 712800, 799200, 885600, 21600, 0 ] +}, { + "tz" : "America/Recife", + "switches" : [ -62135588424, -59006353224, -55850679624, -52695006024, -46383572424, -43227898824, -40072225224, -33760791624, -30605118024, -27449444424, -21138010824, -17982337224, -14826663624, -12219284424, -2208988800 ], + "diffs" : [ -170376, -83976, 2424, 88824, 175224, 261624, 348024, 434424, 520824, 607224, 693624, 780024, 866424, 2424, 0 ] +}, { + "tz" : "America/Regina", + "switches" : [ -62135571684, -59006336484, -55850662884, -52694989284, -46383555684, -43227882084, -40072208484, -33760774884, -30605101284, -27449427684, -21137994084, -17982320484, -14826646884, -12219267684, -2208988800 ], + "diffs" : [ -176316, -89916, -3516, 82884, 169284, 255684, 342084, 428484, 514884, 601284, 687684, 774084, 860484, -3516, 0 ] +}, { + "tz" : "America/Resolute", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -151200, -64800, 21600, 108000, 194400, 280800, 367200, 453600, 540000, 626400, 712800, 799200, 885600, 21600, 0 ] +}, { + "tz" : "America/Rio_Branco", + "switches" : [ -62135580528, -59006345328, -55850671728, -52694998128, -46383564528, -43227890928, -40072217328, -33760783728, -30605110128, -27449436528, -21138002928, -17982329328, -14826655728, -12219276528, -2208988800 ], + "diffs" : [ -171072, -84672, 1728, 88128, 174528, 260928, 347328, 433728, 520128, 606528, 692928, 779328, 865728, 1728, 0 ] +}, { + "tz" : "America/Rosario", + "switches" : [ -62135581392, -59006346192, -55850672592, -52694998992, -46383565392, -43227891792, -40072218192, -33760784592, -30605110992, -27449437392, -21138003792, -17982330192, -14826656592, -12219277392, -2208988800 ], + "diffs" : [ -177408, -91008, -4608, 81792, 168192, 254592, 340992, 427392, 513792, 600192, 686592, 772992, 859392, -4608, 0 ] +}, { + "tz" : "America/Santa_Isabel", + "switches" : [ -62135568716, -59006333516, -55850659916, -52694986316, -46383552716, -43227879116, -40072205516, -33760771916, -30605098316, -27449424716, -21137991116, -17982317516, -14826643916, -12219264716, -2208988800 ], + "diffs" : [ -172084, -85684, 716, 87116, 173516, 259916, 346316, 432716, 519116, 605516, 691916, 778316, 864716, 716, 0 ] +}, { + "tz" : "America/Santarem", + "switches" : [ -62135583672, -59006348472, -55850674872, -52695001272, -46383567672, -43227894072, -40072220472, -33760786872, -30605113272, -27449439672, -21138006072, -17982332472, -14826658872, -12219279672, -2208988800 ], + "diffs" : [ -175128, -88728, -2328, 84072, 170472, 256872, 343272, 429672, 516072, 602472, 688872, 775272, 861672, -2328, 0 ] +}, { + "tz" : "America/Santiago", + "switches" : [ -62135579834, -59006344634, -55850671034, -52694997434, -46383563834, -43227890234, -40072216634, -33760783034, -30605109434, -27449435834, -21138002234, -17982328634, -14826655034, -12219275834, -2208988800 ], + "diffs" : [ -175366, -88966, -2566, 83834, 170234, 256634, 343034, 429434, 515834, 602234, 688634, 775034, 861434, -2566, 0 ] +}, { + "tz" : "America/Santo_Domingo", + "switches" : [ -62135580024, -59006344824, -55850671224, -52694997624, -46383564024, -43227890424, -40072216824, -33760783224, -30605109624, -27449436024, -21138002424, -17982328824, -14826655224, -12219276024, -2524504824, -2208988800 ], + "diffs" : [ -175176, -88776, -2376, 84024, 170424, 256824, 343224, 429624, 516024, 602424, 688824, 775224, 861624, -2376, -2400, 0 ] +}, { + "tz" : "America/Sao_Paulo", + "switches" : [ -62135585612, -59006350412, -55850676812, -52695003212, -46383569612, -43227896012, -40072222412, -33760788812, -30605115212, -27449441612, -21138008012, -17982334412, -14826660812, -12219281612, -2208988800 ], + "diffs" : [ -173188, -86788, -388, 86012, 172412, 258812, 345212, 431612, 518012, 604412, 690812, 777212, 863612, -388, 0 ] +}, { + "tz" : "America/Scoresbysund", + "switches" : [ -62135591528, -59006356328, -55850682728, -52695009128, -46383575528, -43227901928, -40072228328, -33760794728, -30605121128, -27449447528, -21138013928, -17982340328, -14826666728, -12219287528, -2208988800 ], + "diffs" : [ -174472, -88072, -1672, 84728, 171128, 257528, 343928, 430328, 516728, 603128, 689528, 775928, 862328, -1672, 0 ] +}, { + "tz" : "America/Shiprock", + "switches" : [ -62135571604, -59006336404, -55850662804, -52694989204, -46383555604, -43227882004, -40072208404, -33760774804, -30605101204, -27449427604, -21137994004, -17982320404, -14826646804, -12219267604, -2717643600 ], + "diffs" : [ -172796, -86396, 4, 86404, 172804, 259204, 345604, 432004, 518404, 604804, 691204, 777604, 864004, 4, 0 ] +}, { + "tz" : "America/Sitka", + "switches" : [ -62135650727, -59006415527, -55850741927, -52695068327, -46383634727, -43227961127, -40072287527, -33760853927, -30605180327, -27449506727, -21138073127, -17982399527, -14826725927, -12219346727, -3225223727, -2208988800 ], + "diffs" : [ -86473, -73, 86327, 172727, 259127, 345527, 431927, 518327, 604727, 691127, 777527, 863927, 950327, 86327, -73, 0 ] +}, { + "tz" : "America/St_Barthelemy", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/St_Johns", + "switches" : [ -62135584148, -59006348948, -55850675348, -52695001748, -46383568148, -43227894548, -40072220948, -33760787348, -30605113748, -27449440148, -21138006548, -17982332948, -14826659348, -12219280148, -2208988800 ], + "diffs" : [ -172852, -86452, -52, 86348, 172748, 259148, 345548, 431948, 518348, 604748, 691148, 777548, 863948, -52, 0 ] +}, { + "tz" : "America/St_Kitts", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/St_Lucia", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/St_Thomas", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/St_Vincent", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Swift_Current", + "switches" : [ -62135570920, -59006335720, -55850662120, -52694988520, -46383554920, -43227881320, -40072207720, -33760774120, -30605100520, -27449426920, -21137993320, -17982319720, -14826646120, -12219266920, -2208988800 ], + "diffs" : [ -177080, -90680, -4280, 82120, 168520, 254920, 341320, 427720, 514120, 600520, 686920, 773320, 859720, -4280, 0 ] +}, { + "tz" : "America/Tegucigalpa", + "switches" : [ -62135575868, -59006340668, -55850667068, -52694993468, -46383559868, -43227886268, -40072212668, -33760779068, -30605105468, -27449431868, -21137998268, -17982324668, -14826651068, -12219271868, -2208988800 ], + "diffs" : [ -172132, -85732, 668, 87068, 173468, 259868, 346268, 432668, 519068, 605468, 691868, 778268, 864668, 668, 0 ] +}, { + "tz" : "America/Thule", + "switches" : [ -62135580292, -59006345092, -55850671492, -52694997892, -46383564292, -43227890692, -40072217092, -33760783492, -30605109892, -27449436292, -21138002692, -17982329092, -14826655492, -12219276292, -2208988800 ], + "diffs" : [ -174908, -88508, -2108, 84292, 170692, 257092, 343492, 429892, 516292, 602692, 689092, 775492, 861892, -2108, 0 ] +}, { + "tz" : "America/Thunder_Bay", + "switches" : [ -62135575380, -59006340180, -55850666580, -52694992980, -46383559380, -43227885780, -40072212180, -33760778580, -30605104980, -27449431380, -21137997780, -17982324180, -14826650580, -12219271380, -2366733780, -2208988800 ], + "diffs" : [ -176220, -89820, -3420, 82980, 169380, 255780, 342180, 428580, 514980, 601380, 687780, 774180, 860580, -3420, -3600, 0 ] +}, { + "tz" : "America/Tijuana", + "switches" : [ -62135568716, -59006333516, -55850659916, -52694986316, -46383552716, -43227879116, -40072205516, -33760771916, -30605098316, -27449424716, -21137991116, -17982317516, -14826643916, -12219264716, -2208988800 ], + "diffs" : [ -172084, -85684, 716, 87116, 173516, 259916, 346316, 432716, 519116, 605516, 691916, 778316, 864716, 716, 0 ] +}, { + "tz" : "America/Toronto", + "switches" : [ -62135577748, -59006342548, -55850668948, -52694995348, -46383561748, -43227888148, -40072214548, -33760780948, -30605107348, -27449433748, -21138000148, -17982326548, -14826652948, -12219273748, -2366736148 ], + "diffs" : [ -173852, -87452, -1052, 85348, 171748, 258148, 344548, 430948, 517348, 603748, 690148, 776548, 862948, -1052, 0 ] +}, { + "tz" : "America/Tortola", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Vancouver", + "switches" : [ -62135567252, -59006332052, -55850658452, -52694984852, -46383551252, -43227877652, -40072204052, -33760770452, -30605096852, -27449423252, -21137989652, -17982316052, -14826642452, -12219263252, -2713880852 ], + "diffs" : [ -173548, -87148, -748, 85652, 172052, 258452, 344852, 431252, 517652, 604052, 690452, 776852, 863252, -748, 0 ] +}, { + "tz" : "America/Virgin", + "switches" : [ -62135582036, -59006346836, -55850673236, -52694999636, -46383566036, -43227892436, -40072218836, -33760785236, -30605111636, -27449438036, -21138004436, -17982330836, -14826657236, -12219278036, -2208988800 ], + "diffs" : [ -173164, -86764, -364, 86036, 172436, 258836, 345236, 431636, 518036, 604436, 690836, 777236, 863636, -364, 0 ] +}, { + "tz" : "America/Whitehorse", + "switches" : [ -62135564388, -59006329188, -55850655588, -52694981988, -46383548388, -43227874788, -40072201188, -33760767588, -30605093988, -27449420388, -21137986788, -17982313188, -14826639588, -12219260388, -2208988800 ], + "diffs" : [ -176412, -90012, -3612, 82788, 169188, 255588, 341988, 428388, 514788, 601188, 687588, 773988, 860388, -3612, 0 ] +}, { + "tz" : "America/Winnipeg", + "switches" : [ -62135573484, -59006338284, -55850664684, -52694991084, -46383557484, -43227883884, -40072210284, -33760776684, -30605103084, -27449429484, -21137995884, -17982322284, -14826648684, -12219269484, -2602258284 ], + "diffs" : [ -174516, -88116, -1716, 84684, 171084, 257484, 343884, 430284, 516684, 603084, 689484, 775884, 862284, -1716, 0 ] +}, { + "tz" : "America/Yakutat", + "switches" : [ -62135649665, -59006414465, -55850740865, -52695067265, -46383633665, -43227960065, -40072286465, -33760852865, -30605179265, -27449505665, -21138072065, -17982398465, -14826724865, -12219345665, -3225223727, -2208988800 ], + "diffs" : [ -87535, -1135, 85265, 171665, 258065, 344465, 430865, 517265, 603665, 690065, 776465, 862865, 949265, 85265, -1135, 0 ] +}, { + "tz" : "America/Yellowknife", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -147600, -61200, 25200, 111600, 198000, 284400, 370800, 457200, 543600, 630000, 716400, 802800, 889200, 25200, 0 ] +}, { + "tz" : "Antarctica/Casey", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -201600, -115200, -28800, 57600, 144000, 230400, 316800, 403200, 489600, 576000, 662400, 748800, 835200, -28800, 0 ] +}, { + "tz" : "Antarctica/Davis", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -198000, -111600, -25200, 61200, 147600, 234000, 320400, 406800, 493200, 579600, 666000, 752400, 838800, -25200, 0 ] +}, { + "tz" : "Antarctica/DumontDUrville", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -208800, -122400, -36000, 50400, 136800, 223200, 309600, 396000, 482400, 568800, 655200, 741600, 828000, -36000, 0 ] +}, { + "tz" : "Antarctica/Macquarie", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2214259200, -2208988800 ], + "diffs" : [ -212400, -126000, -39600, 46800, 133200, 219600, 306000, 392400, 478800, 565200, 651600, 738000, 824400, -39600, -3600, 0 ] +}, { + "tz" : "Antarctica/Mawson", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -190800, -104400, -18000, 68400, 154800, 241200, 327600, 414000, 500400, 586800, 673200, 759600, 846000, -18000, 0 ] +}, { + "tz" : "Antarctica/McMurdo", + "switches" : [ -62135638744, -59006403544, -55850729944, -52695056344, -46383622744, -43227949144, -40072275544, -33760841944, -30605168344, -27449494744, -21138061144, -17982387544, -14826713944, -12219334744, -3192435544, -2208988800 ], + "diffs" : [ -174056, -87656, -1256, 85144, 171544, 257944, 344344, 430744, 517144, 603544, 689944, 776344, 862744, -1256, -1800, 0 ] +}, { + "tz" : "Antarctica/Palmer", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -162000, -75600, 10800, 97200, 183600, 270000, 356400, 442800, 529200, 615600, 702000, 788400, 874800, 10800, 0 ] +}, { + "tz" : "Antarctica/Rothera", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -162000, -75600, 10800, 97200, 183600, 270000, 356400, 442800, 529200, 615600, 702000, 788400, 874800, 10800, 0 ] +}, { + "tz" : "Antarctica/South_Pole", + "switches" : [ -62135638744, -59006403544, -55850729944, -52695056344, -46383622744, -43227949144, -40072275544, -33760841944, -30605168344, -27449494744, -21138061144, -17982387544, -14826713944, -12219334744, -3192435544, -2208988800 ], + "diffs" : [ -174056, -87656, -1256, 85144, 171544, 257944, 344344, 430744, 517144, 603544, 689944, 776344, 862744, -1256, -1800, 0 ] +}, { + "tz" : "Antarctica/Syowa", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -183600, -97200, -10800, 75600, 162000, 248400, 334800, 421200, 507600, 594000, 680400, 766800, 853200, -10800, 0 ] +}, { + "tz" : "Antarctica/Troll", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Antarctica/Vostok", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -194400, -108000, -21600, 64800, 151200, 237600, 324000, 410400, 496800, 583200, 669600, 756000, 842400, -21600, 0 ] +}, { + "tz" : "Arctic/Longyearbyen", + "switches" : [ -62135599380, -59006364180, -55850690580, -52695016980, -46383583380, -43227909780, -40072236180, -33760802580, -30605128980, -27449455380, -21138021780, -17982348180, -14826674580, -12219295380, -2366757780 ], + "diffs" : [ -173820, -87420, -1020, 85380, 171780, 258180, 344580, 430980, 517380, 603780, 690180, 776580, 862980, -1020, 0 ] +}, { + "tz" : "Asia/Aden", + "switches" : [ -62135608012, -59006372812, -55850699212, -52695025612, -46383592012, -43227918412, -40072244812, -33760811212, -30605137612, -27449464012, -21138030412, -17982356812, -14826683212, -12219304012, -2208988800 ], + "diffs" : [ -172388, -85988, 412, 86812, 173212, 259612, 346012, 432412, 518812, 605212, 691612, 778012, 864412, 412, 0 ] +}, { + "tz" : "Asia/Almaty", + "switches" : [ -62135615268, -59006380068, -55850706468, -52695032868, -46383599268, -43227925668, -40072252068, -33760818468, -30605144868, -27449471268, -21138037668, -17982364068, -14826690468, -12219311268, -2208988800 ], + "diffs" : [ -175932, -89532, -3132, 83268, 169668, 256068, 342468, 428868, 515268, 601668, 688068, 774468, 860868, -3132, 0 ] +}, { + "tz" : "Asia/Amman", + "switches" : [ -62135605424, -59006370224, -55850696624, -52695023024, -46383589424, -43227915824, -40072242224, -33760808624, -30605135024, -27449461424, -21138027824, -17982354224, -14826680624, -12219301424, -2208988800 ], + "diffs" : [ -171376, -84976, 1424, 87824, 174224, 260624, 347024, 433424, 519824, 606224, 692624, 779024, 865424, 1424, 0 ] +}, { + "tz" : "Asia/Anadyr", + "switches" : [ -62135639396, -59006404196, -55850730596, -52695056996, -46383623396, -43227949796, -40072276196, -33760842596, -30605168996, -27449495396, -21138061796, -17982388196, -14826714596, -12219335396, -2208988800 ], + "diffs" : [ -173404, -87004, -604, 85796, 172196, 258596, 344996, 431396, 517796, 604196, 690596, 776996, 863396, -604, 0 ] +}, { + "tz" : "Asia/Aqtau", + "switches" : [ -62135608864, -59006373664, -55850700064, -52695026464, -46383592864, -43227919264, -40072245664, -33760812064, -30605138464, -27449464864, -21138031264, -17982357664, -14826684064, -12219304864, -2208988800 ], + "diffs" : [ -178736, -92336, -5936, 80464, 166864, 253264, 339664, 426064, 512464, 598864, 685264, 771664, 858064, -5936, 0 ] +}, { + "tz" : "Asia/Aqtobe", + "switches" : [ -62135610520, -59006375320, -55850701720, -52695028120, -46383594520, -43227920920, -40072247320, -33760813720, -30605140120, -27449466520, -21138032920, -17982359320, -14826685720, -12219306520, -2208988800 ], + "diffs" : [ -177080, -90680, -4280, 82120, 168520, 254920, 341320, 427720, 514120, 600520, 686920, 773320, 859720, -4280, 0 ] +}, { + "tz" : "Asia/Ashgabat", + "switches" : [ -62135610812, -59006375612, -55850702012, -52695028412, -46383594812, -43227921212, -40072247612, -33760814012, -30605140412, -27449466812, -21138033212, -17982359612, -14826686012, -12219306812, -2208988800 ], + "diffs" : [ -176788, -90388, -3988, 82412, 168812, 255212, 341612, 428012, 514412, 600812, 687212, 773612, 860012, -3988, 0 ] +}, { + "tz" : "Asia/Ashkhabad", + "switches" : [ -62135610812, -59006375612, -55850702012, -52695028412, -46383594812, -43227921212, -40072247612, -33760814012, -30605140412, -27449466812, -21138033212, -17982359612, -14826686012, -12219306812, -2208988800 ], + "diffs" : [ -176788, -90388, -3988, 82412, 168812, 255212, 341612, 428012, 514412, 600812, 687212, 773612, 860012, -3988, 0 ] +}, { + "tz" : "Asia/Atyrau", + "switches" : [ -62135609264, -59006374064, -55850700464, -52695026864, -46383593264, -43227919664, -40072246064, -33760812464, -30605138864, -27449465264, -21138031664, -17982358064, -14826684464, -12219305264, -2208988800 ], + "diffs" : [ -178336, -91936, -5536, 80864, 167264, 253664, 340064, 426464, 512864, 599264, 685664, 772064, 858464, -5536, 0 ] +}, { + "tz" : "Asia/Baghdad", + "switches" : [ -62135607460, -59006372260, -55850698660, -52695025060, -46383591460, -43227917860, -40072244260, -33760810660, -30605137060, -27449463460, -21138029860, -17982356260, -14826682660, -12219303460, -2524532260, -2208988800 ], + "diffs" : [ -172940, -86540, -140, 86260, 172660, 259060, 345460, 431860, 518260, 604660, 691060, 777460, 863860, -140, -144, 0 ] +}, { + "tz" : "Asia/Bahrain", + "switches" : [ -62135609168, -59006373968, -55850700368, -52695026768, -46383593168, -43227919568, -40072245968, -33760812368, -30605138768, -27449465168, -21138031568, -17982357968, -14826684368, -12219305168, -2208988800 ], + "diffs" : [ -171232, -84832, 1568, 87968, 174368, 260768, 347168, 433568, 519968, 606368, 692768, 779168, 865568, 1568, 0 ] +}, { + "tz" : "Asia/Baku", + "switches" : [ -62135608764, -59006373564, -55850699964, -52695026364, -46383592764, -43227919164, -40072245564, -33760811964, -30605138364, -27449464764, -21138031164, -17982357564, -14826683964, -12219304764, -2208988800 ], + "diffs" : [ -175236, -88836, -2436, 83964, 170364, 256764, 343164, 429564, 515964, 602364, 688764, 775164, 861564, -2436, 0 ] +}, { + "tz" : "Asia/Bangkok", + "switches" : [ -62135620924, -59006385724, -55850712124, -52695038524, -46383604924, -43227931324, -40072257724, -33760824124, -30605150524, -27449476924, -21138043324, -17982369724, -14826696124, -12219316924, -2208988800 ], + "diffs" : [ -173876, -87476, -1076, 85324, 171724, 258124, 344524, 430924, 517324, 603724, 690124, 776524, 862924, -1076, 0 ] +}, { + "tz" : "Asia/Barnaul", + "switches" : [ -62135616900, -59006381700, -55850708100, -52695034500, -46383600900, -43227927300, -40072253700, -33760820100, -30605146500, -27449472900, -21138039300, -17982365700, -14826692100, -12219312900, -2208988800 ], + "diffs" : [ -177900, -91500, -5100, 81300, 167700, 254100, 340500, 426900, 513300, 599700, 686100, 772500, 858900, -5100, 0 ] +}, { + "tz" : "Asia/Beirut", + "switches" : [ -62135605320, -59006370120, -55850696520, -52695022920, -46383589320, -43227915720, -40072242120, -33760808520, -30605134920, -27449461320, -21138027720, -17982354120, -14826680520, -12219301320, -2840149320 ], + "diffs" : [ -171480, -85080, 1320, 87720, 174120, 260520, 346920, 433320, 519720, 606120, 692520, 778920, 865320, 1320, 0 ] +}, { + "tz" : "Asia/Bishkek", + "switches" : [ -62135614704, -59006379504, -55850705904, -52695032304, -46383598704, -43227925104, -40072251504, -33760817904, -30605144304, -27449470704, -21138037104, -17982363504, -14826689904, -12219310704, -2208988800 ], + "diffs" : [ -176496, -90096, -3696, 82704, 169104, 255504, 341904, 428304, 514704, 601104, 687504, 773904, 860304, -3696, 0 ] +}, { + "tz" : "Asia/Brunei", + "switches" : [ -62135624380, -59006389180, -55850715580, -52695041980, -46383608380, -43227934780, -40072261180, -33760827580, -30605153980, -27449480380, -21138046780, -17982373180, -14826699580, -12219320380, -2208988800 ], + "diffs" : [ -174020, -87620, -1220, 85180, 171580, 257980, 344380, 430780, 517180, 603580, 689980, 776380, 862780, -1220, 0 ] +}, { + "tz" : "Asia/Calcutta", + "switches" : [ -62135618008, -59006382808, -55850709208, -52695035608, -46383602008, -43227928408, -40072254808, -33760821208, -30605147608, -27449474008, -21138040408, -17982366808, -14826693208, -12219314008, -3645237208, -3155694800, -2208988800 ], + "diffs" : [ -171392, -84992, 1408, 87808, 174208, 260608, 347008, 433408, 519808, 606208, 692608, 779008, 865408, 1408, 1400, -530, 0 ] +}, { + "tz" : "Asia/Chita", + "switches" : [ -62135624032, -59006388832, -55850715232, -52695041632, -46383608032, -43227934432, -40072260832, -33760827232, -30605153632, -27449480032, -21138046432, -17982372832, -14826699232, -12219320032, -2208988800 ], + "diffs" : [ -177968, -91568, -5168, 81232, 167632, 254032, 340432, 426832, 513232, 599632, 686032, 772432, 858832, -5168, 0 ] +}, { + "tz" : "Asia/Choibalsan", + "switches" : [ -62135624280, -59006389080, -55850715480, -52695041880, -46383608280, -43227934680, -40072261080, -33760827480, -30605153880, -27449480280, -21138046680, -17982373080, -14826699480, -12219320280, -2208988800 ], + "diffs" : [ -174120, -87720, -1320, 85080, 171480, 257880, 344280, 430680, 517080, 603480, 689880, 776280, 862680, -1320, 0 ] +}, { + "tz" : "Asia/Chongqing", + "switches" : [ -62135625943, -59006390743, -55850717143, -52695043543, -46383609943, -43227936343, -40072262743, -33760829143, -30605155543, -27449481943, -21138048343, -17982374743, -14826701143, -12219321943, -2208988800 ], + "diffs" : [ -172457, -86057, 343, 86743, 173143, 259543, 345943, 432343, 518743, 605143, 691543, 777943, 864343, 343, 0 ] +}, { + "tz" : "Asia/Chungking", + "switches" : [ -62135625943, -59006390743, -55850717143, -52695043543, -46383609943, -43227936343, -40072262743, -33760829143, -30605155543, -27449481943, -21138048343, -17982374743, -14826701143, -12219321943, -2208988800 ], + "diffs" : [ -172457, -86057, 343, 86743, 173143, 259543, 345943, 432343, 518743, 605143, 691543, 777943, 864343, 343, 0 ] +}, { + "tz" : "Asia/Colombo", + "switches" : [ -62135615964, -59006380764, -55850707164, -52695033564, -46383599964, -43227926364, -40072252764, -33760819164, -30605145564, -27449471964, -21138038364, -17982364764, -14826691164, -12219311964, -2840159964, -2208988800 ], + "diffs" : [ -173436, -87036, -636, 85764, 172164, 258564, 344964, 431364, 517764, 604164, 690564, 776964, 863364, -636, -628, 0 ] +}, { + "tz" : "Asia/Dacca", + "switches" : [ -62135618500, -59006383300, -55850709700, -52695036100, -46383602500, -43227928900, -40072255300, -33760821700, -30605148100, -27449474500, -21138040900, -17982367300, -14826693700, -12219314500, -2524543300, -2208988800 ], + "diffs" : [ -172700, -86300, 100, 86500, 172900, 259300, 345700, 432100, 518500, 604900, 691300, 777700, 864100, 100, -400, 0 ] +}, { + "tz" : "Asia/Damascus", + "switches" : [ -62135605512, -59006370312, -55850696712, -52695023112, -46383589512, -43227915912, -40072242312, -33760808712, -30605135112, -27449461512, -21138027912, -17982354312, -14826680712, -12219301512, -2208988800 ], + "diffs" : [ -171288, -84888, 1512, 87912, 174312, 260712, 347112, 433512, 519912, 606312, 692712, 779112, 865512, 1512, 0 ] +}, { + "tz" : "Asia/Dhaka", + "switches" : [ -62135618500, -59006383300, -55850709700, -52695036100, -46383602500, -43227928900, -40072255300, -33760821700, -30605148100, -27449474500, -21138040900, -17982367300, -14826693700, -12219314500, -2524543300, -2208988800 ], + "diffs" : [ -172700, -86300, 100, 86500, 172900, 259300, 345700, 432100, 518500, 604900, 691300, 777700, 864100, 100, -400, 0 ] +}, { + "tz" : "Asia/Dili", + "switches" : [ -62135626940, -59006391740, -55850718140, -52695044540, -46383610940, -43227937340, -40072263740, -33760830140, -30605156540, -27449482940, -21138049340, -17982375740, -14826702140, -12219322940, -2208988800 ], + "diffs" : [ -175060, -88660, -2260, 84140, 170540, 256940, 343340, 429740, 516140, 602540, 688940, 775340, 861740, -2260, 0 ] +}, { + "tz" : "Asia/Dubai", + "switches" : [ -62135610072, -59006374872, -55850701272, -52695027672, -46383594072, -43227920472, -40072246872, -33760813272, -30605139672, -27449466072, -21138032472, -17982358872, -14826685272, -12219306072, -2208988800 ], + "diffs" : [ -173928, -87528, -1128, 85272, 171672, 258072, 344472, 430872, 517272, 603672, 690072, 776472, 862872, -1128, 0 ] +}, { + "tz" : "Asia/Dushanbe", + "switches" : [ -62135613312, -59006378112, -55850704512, -52695030912, -46383597312, -43227923712, -40072250112, -33760816512, -30605142912, -27449469312, -21138035712, -17982362112, -14826688512, -12219309312, -2208988800 ], + "diffs" : [ -174288, -87888, -1488, 84912, 171312, 257712, 344112, 430512, 516912, 603312, 689712, 776112, 862512, -1488, 0 ] +}, { + "tz" : "Asia/Famagusta", + "switches" : [ -62135604948, -59006369748, -55850696148, -52695022548, -46383588948, -43227915348, -40072241748, -33760808148, -30605134548, -27449460948, -21138027348, -17982353748, -14826680148, -12219300948, -2208988800 ], + "diffs" : [ -171852, -85452, 948, 87348, 173748, 260148, 346548, 432948, 519348, 605748, 692148, 778548, 864948, 948, 0 ] +}, { + "tz" : "Asia/Gaza", + "switches" : [ -62135605072, -59006369872, -55850696272, -52695022672, -46383589072, -43227915472, -40072241872, -33760808272, -30605134672, -27449461072, -21138027472, -17982353872, -14826680272, -12219301072, -2208988800 ], + "diffs" : [ -171728, -85328, 1072, 87472, 173872, 260272, 346672, 433072, 519472, 605872, 692272, 778672, 865072, 1072, 0 ] +}, { + "tz" : "Asia/Harbin", + "switches" : [ -62135625943, -59006390743, -55850717143, -52695043543, -46383609943, -43227936343, -40072262743, -33760829143, -30605155543, -27449481943, -21138048343, -17982374743, -14826701143, -12219321943, -2208988800 ], + "diffs" : [ -172457, -86057, 343, 86743, 173143, 259543, 345943, 432343, 518743, 605143, 691543, 777943, 864343, 343, 0 ] +}, { + "tz" : "Asia/Hebron", + "switches" : [ -62135605223, -59006370023, -55850696423, -52695022823, -46383589223, -43227915623, -40072242023, -33760808423, -30605134823, -27449461223, -21138027623, -17982354023, -14826680423, -12219301223, -2208988800 ], + "diffs" : [ -171577, -85177, 1223, 87623, 174023, 260423, 346823, 433223, 519623, 606023, 692423, 778823, 865223, 1223, 0 ] +}, { + "tz" : "Asia/Ho_Chi_Minh", + "switches" : [ -62135622400, -59006387200, -55850713600, -52695040000, -46383606400, -43227932800, -40072259200, -33760825600, -30605152000, -27449478400, -21138044800, -17982371200, -14826697600, -12219318400, -2208988800 ], + "diffs" : [ -172400, -86000, 400, 86800, 173200, 259600, 346000, 432400, 518800, 605200, 691600, 778000, 864400, 400, 0 ] +}, { + "tz" : "Asia/Hong_Kong", + "switches" : [ -62135624202, -59006389002, -55850715402, -52695041802, -46383608202, -43227934602, -40072261002, -33760827402, -30605153802, -27449480202, -21138046602, -17982373002, -14826699402, -12219320202, -2208988800 ], + "diffs" : [ -174198, -87798, -1398, 85002, 171402, 257802, 344202, 430602, 517002, 603402, 689802, 776202, 862602, -1398, 0 ] +}, { + "tz" : "Asia/Hovd", + "switches" : [ -62135618796, -59006383596, -55850709996, -52695036396, -46383602796, -43227929196, -40072255596, -33760821996, -30605148396, -27449474796, -21138041196, -17982367596, -14826693996, -12219314796, -2208988800 ], + "diffs" : [ -176004, -89604, -3204, 83196, 169596, 255996, 342396, 428796, 515196, 601596, 687996, 774396, 860796, -3204, 0 ] +}, { + "tz" : "Asia/Irkutsk", + "switches" : [ -62135621825, -59006386625, -55850713025, -52695039425, -46383605825, -43227932225, -40072258625, -33760825025, -30605151425, -27449477825, -21138044225, -17982370625, -14826697025, -12219317825, -2208988800 ], + "diffs" : [ -176575, -90175, -3775, 82625, 169025, 255425, 341825, 428225, 514625, 601025, 687425, 773825, 860225, -3775, 0 ] +}, { + "tz" : "Asia/Istanbul", + "switches" : [ -62135603752, -59006368552, -55850694952, -52695021352, -46383587752, -43227914152, -40072240552, -33760806952, -30605133352, -27449459752, -21138026152, -17982352552, -14826678952, -12219299752, -2840147752, -2208988800 ], + "diffs" : [ -176648, -90248, -3848, 82552, 168952, 255352, 341752, 428152, 514552, 600952, 687352, 773752, 860152, -3848, -3784, 0 ] +}, { + "tz" : "Asia/Jakarta", + "switches" : [ -62135622432, -59006387232, -55850713632, -52695040032, -46383606432, -43227932832, -40072259232, -33760825632, -30605152032, -27449478432, -21138044832, -17982371232, -14826697632, -12219318432, -2208988800 ], + "diffs" : [ -172368, -85968, 432, 86832, 173232, 259632, 346032, 432432, 518832, 605232, 691632, 778032, 864432, 432, 0 ] +}, { + "tz" : "Asia/Jayapura", + "switches" : [ -62135630568, -59006395368, -55850721768, -52695048168, -46383614568, -43227940968, -40072267368, -33760833768, -30605160168, -27449486568, -21138052968, -17982379368, -14826705768, -12219326568, -2208988800 ], + "diffs" : [ -171432, -85032, 1368, 87768, 174168, 260568, 346968, 433368, 519768, 606168, 692568, 778968, 865368, 1368, 0 ] +}, { + "tz" : "Asia/Jerusalem", + "switches" : [ -62135605254, -59006370054, -55850696454, -52695022854, -46383589254, -43227915654, -40072242054, -33760808454, -30605134854, -27449461254, -21138027654, -17982354054, -14826680454, -12219301254, -2840149254, -2208988800 ], + "diffs" : [ -171546, -85146, 1254, 87654, 174054, 260454, 346854, 433254, 519654, 606054, 692454, 778854, 865254, 1254, 1240, 0 ] +}, { + "tz" : "Asia/Kabul", + "switches" : [ -62135613408, -59006378208, -55850704608, -52695031008, -46383597408, -43227923808, -40072250208, -33760816608, -30605143008, -27449469408, -21138035808, -17982362208, -14826688608, -12219309408, -2524538208, -2208988800 ], + "diffs" : [ -172392, -85992, 408, 86808, 173208, 259608, 346008, 432408, 518808, 605208, 691608, 778008, 864408, 408, -1800, 0 ] +}, { + "tz" : "Asia/Kamchatka", + "switches" : [ -62135634876, -59006399676, -55850726076, -52695052476, -46383618876, -43227945276, -40072271676, -33760838076, -30605164476, -27449490876, -21138057276, -17982383676, -14826710076, -12219330876, -2208988800 ], + "diffs" : [ -177924, -91524, -5124, 81276, 167676, 254076, 340476, 426876, 513276, 599676, 686076, 772476, 858876, -5124, 0 ] +}, { + "tz" : "Asia/Karachi", + "switches" : [ -62135612892, -59006377692, -55850704092, -52695030492, -46383596892, -43227923292, -40072249692, -33760816092, -30605142492, -27449468892, -21138035292, -17982361692, -14826688092, -12219308892, -2208988800 ], + "diffs" : [ -174708, -88308, -1908, 84492, 170892, 257292, 343692, 430092, 516492, 602892, 689292, 775692, 862092, -1908, 0 ] +}, { + "tz" : "Asia/Kashgar", + "switches" : [ -62135617820, -59006382620, -55850709020, -52695035420, -46383601820, -43227928220, -40072254620, -33760821020, -30605147420, -27449473820, -21138040220, -17982366620, -14826693020, -12219313820, -2208988800 ], + "diffs" : [ -173380, -86980, -580, 85820, 172220, 258620, 345020, 431420, 517820, 604220, 690620, 777020, 863420, -580, 0 ] +}, { + "tz" : "Asia/Kathmandu", + "switches" : [ -62135617276, -59006382076, -55850708476, -52695034876, -46383601276, -43227927676, -40072254076, -33760820476, -30605146876, -27449473276, -21138039676, -17982366076, -14826692476, -12219313276, -2208988800 ], + "diffs" : [ -173024, -86624, -224, 86176, 172576, 258976, 345376, 431776, 518176, 604576, 690976, 777376, 863776, -224, 0 ] +}, { + "tz" : "Asia/Katmandu", + "switches" : [ -62135617276, -59006382076, -55850708476, -52695034876, -46383601276, -43227927676, -40072254076, -33760820476, -30605146876, -27449473276, -21138039676, -17982366076, -14826692476, -12219313276, -2208988800 ], + "diffs" : [ -173024, -86624, -224, 86176, 172576, 258976, 345376, 431776, 518176, 604576, 690976, 777376, 863776, -224, 0 ] +}, { + "tz" : "Asia/Khandyga", + "switches" : [ -62135629333, -59006394133, -55850720533, -52695046933, -46383613333, -43227939733, -40072266133, -33760832533, -30605158933, -27449485333, -21138051733, -17982378133, -14826704533, -12219325333, -2208988800 ], + "diffs" : [ -172667, -86267, 133, 86533, 172933, 259333, 345733, 432133, 518533, 604933, 691333, 777733, 864133, 133, 0 ] +}, { + "tz" : "Asia/Kolkata", + "switches" : [ -62135618008, -59006382808, -55850709208, -52695035608, -46383602008, -43227928408, -40072254808, -33760821208, -30605147608, -27449474008, -21138040408, -17982366808, -14826693208, -12219314008, -3645237208, -3155694800, -2208988800 ], + "diffs" : [ -171392, -84992, 1408, 87808, 174208, 260608, 347008, 433408, 519808, 606208, 692608, 779008, 865408, 1408, 1400, -530, 0 ] +}, { + "tz" : "Asia/Krasnoyarsk", + "switches" : [ -62135619086, -59006383886, -55850710286, -52695036686, -46383603086, -43227929486, -40072255886, -33760822286, -30605148686, -27449475086, -21138041486, -17982367886, -14826694286, -12219315086, -2208988800 ], + "diffs" : [ -175714, -89314, -2914, 83486, 169886, 256286, 342686, 429086, 515486, 601886, 688286, 774686, 861086, -2914, 0 ] +}, { + "tz" : "Asia/Kuala_Lumpur", + "switches" : [ -62135621206, -59006386006, -55850712406, -52695038806, -46383605206, -43227931606, -40072258006, -33760824406, -30605150806, -27449477206, -21138043606, -17982370006, -14826696406, -12219317206, -2208988800 ], + "diffs" : [ -177194, -90794, -4394, 82006, 168406, 254806, 341206, 427606, 514006, 600406, 686806, 773206, 859606, -4394, 0 ] +}, { + "tz" : "Asia/Kuching", + "switches" : [ -62135623280, -59006388080, -55850714480, -52695040880, -46383607280, -43227933680, -40072260080, -33760826480, -30605152880, -27449479280, -21138045680, -17982372080, -14826698480, -12219319280, -2208988800 ], + "diffs" : [ -175120, -88720, -2320, 84080, 170480, 256880, 343280, 429680, 516080, 602480, 688880, 775280, 861680, -2320, 0 ] +}, { + "tz" : "Asia/Kuwait", + "switches" : [ -62135608012, -59006372812, -55850699212, -52695025612, -46383592012, -43227918412, -40072244812, -33760811212, -30605137612, -27449464012, -21138030412, -17982356812, -14826683212, -12219304012, -2208988800 ], + "diffs" : [ -172388, -85988, 412, 86812, 173212, 259612, 346012, 432412, 518812, 605212, 691612, 778012, 864412, 412, 0 ] +}, { + "tz" : "Asia/Macao", + "switches" : [ -62135624050, -59006388850, -55850715250, -52695041650, -46383608050, -43227934450, -40072260850, -33760827250, -30605153650, -27449480050, -21138046450, -17982372850, -14826699250, -12219320050, -2208988800 ], + "diffs" : [ -174350, -87950, -1550, 84850, 171250, 257650, 344050, 430450, 516850, 603250, 689650, 776050, 862450, -1550, 0 ] +}, { + "tz" : "Asia/Macau", + "switches" : [ -62135624050, -59006388850, -55850715250, -52695041650, -46383608050, -43227934450, -40072260850, -33760827250, -30605153650, -27449480050, -21138046450, -17982372850, -14826699250, -12219320050, -2208988800 ], + "diffs" : [ -174350, -87950, -1550, 84850, 171250, 257650, 344050, 430450, 516850, 603250, 689650, 776050, 862450, -1550, 0 ] +}, { + "tz" : "Asia/Magadan", + "switches" : [ -62135632992, -59006397792, -55850724192, -52695050592, -46383616992, -43227943392, -40072269792, -33760836192, -30605162592, -27449488992, -21138055392, -17982381792, -14826708192, -12219328992, -2208988800 ], + "diffs" : [ -176208, -89808, -3408, 82992, 169392, 255792, 342192, 428592, 514992, 601392, 687792, 774192, 860592, -3408, 0 ] +}, { + "tz" : "Asia/Makassar", + "switches" : [ -62135625456, -59006390256, -55850716656, -52695043056, -46383609456, -43227935856, -40072262256, -33760828656, -30605155056, -27449481456, -21138047856, -17982374256, -14826700656, -12219321456, -2208988800 ], + "diffs" : [ -172944, -86544, -144, 86256, 172656, 259056, 345456, 431856, 518256, 604656, 691056, 777456, 863856, -144, 0 ] +}, { + "tz" : "Asia/Manila", + "switches" : [ -62135539440, -59006304240, -55850630640, -52694957040, -46383523440, -43227849840, -40072176240, -33760742640, -30605069040, -27449395440, -21137961840, -17982288240, -14826614640, -12219235440, -3944621040, -2229321840 ], + "diffs" : [ -258960, -172560, -86160, 240, 86640, 173040, 259440, 345840, 432240, 518640, 605040, 691440, 777840, -86160, 240, 0 ] +}, { + "tz" : "Asia/Muscat", + "switches" : [ -62135610072, -59006374872, -55850701272, -52695027672, -46383594072, -43227920472, -40072246872, -33760813272, -30605139672, -27449466072, -21138032472, -17982358872, -14826685272, -12219306072, -2208988800 ], + "diffs" : [ -173928, -87528, -1128, 85272, 171672, 258072, 344472, 430872, 517272, 603672, 690072, 776472, 862872, -1128, 0 ] +}, { + "tz" : "Asia/Nicosia", + "switches" : [ -62135604808, -59006369608, -55850696008, -52695022408, -46383588808, -43227915208, -40072241608, -33760808008, -30605134408, -27449460808, -21138027208, -17982353608, -14826680008, -12219300808, -2208988800 ], + "diffs" : [ -171992, -85592, 808, 87208, 173608, 260008, 346408, 432808, 519208, 605608, 692008, 778408, 864808, 808, 0 ] +}, { + "tz" : "Asia/Novokuznetsk", + "switches" : [ -62135617728, -59006382528, -55850708928, -52695035328, -46383601728, -43227928128, -40072254528, -33760820928, -30605147328, -27449473728, -21138040128, -17982366528, -14826692928, -12219313728, -2208988800 ], + "diffs" : [ -177072, -90672, -4272, 82128, 168528, 254928, 341328, 427728, 514128, 600528, 686928, 773328, 859728, -4272, 0 ] +}, { + "tz" : "Asia/Novosibirsk", + "switches" : [ -62135616700, -59006381500, -55850707900, -52695034300, -46383600700, -43227927100, -40072253500, -33760819900, -30605146300, -27449472700, -21138039100, -17982365500, -14826691900, -12219312700, -2208988800 ], + "diffs" : [ -178100, -91700, -5300, 81100, 167500, 253900, 340300, 426700, 513100, 599500, 685900, 772300, 858700, -5300, 0 ] +}, { + "tz" : "Asia/Omsk", + "switches" : [ -62135614410, -59006379210, -55850705610, -52695032010, -46383598410, -43227924810, -40072251210, -33760817610, -30605144010, -27449470410, -21138036810, -17982363210, -14826689610, -12219310410, -2208988800 ], + "diffs" : [ -176790, -90390, -3990, 82410, 168810, 255210, 341610, 428010, 514410, 600810, 687210, 773610, 860010, -3990, 0 ] +}, { + "tz" : "Asia/Oral", + "switches" : [ -62135609124, -59006373924, -55850700324, -52695026724, -46383593124, -43227919524, -40072245924, -33760812324, -30605138724, -27449465124, -21138031524, -17982357924, -14826684324, -12219305124, -2208988800 ], + "diffs" : [ -178476, -92076, -5676, 80724, 167124, 253524, 339924, 426324, 512724, 599124, 685524, 771924, 858324, -5676, 0 ] +}, { + "tz" : "Asia/Phnom_Penh", + "switches" : [ -62135620924, -59006385724, -55850712124, -52695038524, -46383604924, -43227931324, -40072257724, -33760824124, -30605150524, -27449476924, -21138043324, -17982369724, -14826696124, -12219316924, -2208988800 ], + "diffs" : [ -173876, -87476, -1076, 85324, 171724, 258124, 344524, 430924, 517324, 603724, 690124, 776524, 862924, -1076, 0 ] +}, { + "tz" : "Asia/Pontianak", + "switches" : [ -62135623040, -59006387840, -55850714240, -52695040640, -46383607040, -43227933440, -40072259840, -33760826240, -30605152640, -27449479040, -21138045440, -17982371840, -14826698240, -12219319040, -2208988800 ], + "diffs" : [ -171760, -85360, 1040, 87440, 173840, 260240, 346640, 433040, 519440, 605840, 692240, 778640, 865040, 1040, 0 ] +}, { + "tz" : "Asia/Pyongyang", + "switches" : [ -62135626980, -59006391780, -55850718180, -52695044580, -46383610980, -43227937380, -40072263780, -33760830180, -30605156580, -27449482980, -21138049380, -17982375780, -14826702180, -12219322980, -2208988800 ], + "diffs" : [ -175020, -88620, -2220, 84180, 170580, 256980, 343380, 429780, 516180, 602580, 688980, 775380, 861780, -2220, 0 ] +}, { + "tz" : "Asia/Qatar", + "switches" : [ -62135609168, -59006373968, -55850700368, -52695026768, -46383593168, -43227919568, -40072245968, -33760812368, -30605138768, -27449465168, -21138031568, -17982357968, -14826684368, -12219305168, -2208988800 ], + "diffs" : [ -171232, -84832, 1568, 87968, 174368, 260768, 347168, 433568, 519968, 606368, 692768, 779168, 865568, 1568, 0 ] +}, { + "tz" : "Asia/Qostanay", + "switches" : [ -62135612068, -59006376868, -55850703268, -52695029668, -46383596068, -43227922468, -40072248868, -33760815268, -30605141668, -27449468068, -21138034468, -17982360868, -14826687268, -12219308068, -2208988800 ], + "diffs" : [ -179132, -92732, -6332, 80068, 166468, 252868, 339268, 425668, 512068, 598468, 684868, 771268, 857668, -6332, 0 ] +}, { + "tz" : "Asia/Qyzylorda", + "switches" : [ -62135612512, -59006377312, -55850703712, -52695030112, -46383596512, -43227922912, -40072249312, -33760815712, -30605142112, -27449468512, -21138034912, -17982361312, -14826687712, -12219308512, -2208988800 ], + "diffs" : [ -175088, -88688, -2288, 84112, 170512, 256912, 343312, 429712, 516112, 602512, 688912, 775312, 861712, -2288, 0 ] +}, { + "tz" : "Asia/Rangoon", + "switches" : [ -62135619887, -59006384687, -55850711087, -52695037487, -46383603887, -43227930287, -40072256687, -33760823087, -30605149487, -27449475887, -21138042287, -17982368687, -14826695087, -12219315887, -2208988800 ], + "diffs" : [ -173113, -86713, -313, 86087, 172487, 258887, 345287, 431687, 518087, 604487, 690887, 777287, 863687, -313, 0 ] +}, { + "tz" : "Asia/Riyadh", + "switches" : [ -62135608012, -59006372812, -55850699212, -52695025612, -46383592012, -43227918412, -40072244812, -33760811212, -30605137612, -27449464012, -21138030412, -17982356812, -14826683212, -12219304012, -2208988800 ], + "diffs" : [ -172388, -85988, 412, 86812, 173212, 259612, 346012, 432412, 518812, 605212, 691612, 778012, 864412, 412, 0 ] +}, { + "tz" : "Asia/Saigon", + "switches" : [ -62135622400, -59006387200, -55850713600, -52695040000, -46383606400, -43227932800, -40072259200, -33760825600, -30605152000, -27449478400, -21138044800, -17982371200, -14826697600, -12219318400, -2208988800 ], + "diffs" : [ -172400, -86000, 400, 86800, 173200, 259600, 346000, 432400, 518800, 605200, 691600, 778000, 864400, 400, 0 ] +}, { + "tz" : "Asia/Sakhalin", + "switches" : [ -62135631048, -59006395848, -55850722248, -52695048648, -46383615048, -43227941448, -40072267848, -33760834248, -30605160648, -27449487048, -21138053448, -17982379848, -14826706248, -12219327048, -2208988800 ], + "diffs" : [ -178152, -91752, -5352, 81048, 167448, 253848, 340248, 426648, 513048, 599448, 685848, 772248, 858648, -5352, 0 ] +}, { + "tz" : "Asia/Samarkand", + "switches" : [ -62135612873, -59006377673, -55850704073, -52695030473, -46383596873, -43227923273, -40072249673, -33760816073, -30605142473, -27449468873, -21138035273, -17982361673, -14826688073, -12219308873, -2208988800 ], + "diffs" : [ -174727, -88327, -1927, 84473, 170873, 257273, 343673, 430073, 516473, 602873, 689273, 775673, 862073, -1927, 0 ] +}, { + "tz" : "Asia/Seoul", + "switches" : [ -62135627272, -59006392072, -55850718472, -52695044872, -46383611272, -43227937672, -40072264072, -33760830472, -30605156872, -27449483272, -21138049672, -17982376072, -14826702472, -12219323272, -2208988800 ], + "diffs" : [ -174728, -88328, -1928, 84472, 170872, 257272, 343672, 430072, 516472, 602872, 689272, 775672, 862072, -1928, 0 ] +}, { + "tz" : "Asia/Shanghai", + "switches" : [ -62135625943, -59006390743, -55850717143, -52695043543, -46383609943, -43227936343, -40072262743, -33760829143, -30605155543, -27449481943, -21138048343, -17982374743, -14826701143, -12219321943, -2208988800 ], + "diffs" : [ -172457, -86057, 343, 86743, 173143, 259543, 345943, 432343, 518743, 605143, 691543, 777943, 864343, 343, 0 ] +}, { + "tz" : "Asia/Singapore", + "switches" : [ -62135621725, -59006386525, -55850712925, -52695039325, -46383605725, -43227932125, -40072258525, -33760824925, -30605151325, -27449477725, -21138044125, -17982370525, -14826696925, -12219317725, -2208988800 ], + "diffs" : [ -176675, -90275, -3875, 82525, 168925, 255325, 341725, 428125, 514525, 600925, 687325, 773725, 860125, -3875, 0 ] +}, { + "tz" : "Asia/Srednekolymsk", + "switches" : [ -62135633692, -59006398492, -55850724892, -52695051292, -46383617692, -43227944092, -40072270492, -33760836892, -30605163292, -27449489692, -21138056092, -17982382492, -14826708892, -12219329692, -2208988800 ], + "diffs" : [ -175508, -89108, -2708, 83692, 170092, 256492, 342892, 429292, 515692, 602092, 688492, 774892, 861292, -2708, 0 ] +}, { + "tz" : "Asia/Taipei", + "switches" : [ -62135625960, -59006390760, -55850717160, -52695043560, -46383609960, -43227936360, -40072262760, -33760829160, -30605155560, -27449481960, -21138048360, -17982374760, -14826701160, -12219321960, -2335248360 ], + "diffs" : [ -172440, -86040, 360, 86760, 173160, 259560, 345960, 432360, 518760, 605160, 691560, 777960, 864360, 360, 0 ] +}, { + "tz" : "Asia/Tashkent", + "switches" : [ -62135613431, -59006378231, -55850704631, -52695031031, -46383597431, -43227923831, -40072250231, -33760816631, -30605143031, -27449469431, -21138035831, -17982362231, -14826688631, -12219309431, -2208988800 ], + "diffs" : [ -174169, -87769, -1369, 85031, 171431, 257831, 344231, 430631, 517031, 603431, 689831, 776231, 862631, -1369, 0 ] +}, { + "tz" : "Asia/Tbilisi", + "switches" : [ -62135607551, -59006372351, -55850698751, -52695025151, -46383591551, -43227917951, -40072244351, -33760810751, -30605137151, -27449463551, -21138029951, -17982356351, -14826682751, -12219303551, -2208988800 ], + "diffs" : [ -176449, -90049, -3649, 82751, 169151, 255551, 341951, 428351, 514751, 601151, 687551, 773951, 860351, -3649, 0 ] +}, { + "tz" : "Asia/Tel_Aviv", + "switches" : [ -62135605254, -59006370054, -55850696454, -52695022854, -46383589254, -43227915654, -40072242054, -33760808454, -30605134854, -27449461254, -21138027654, -17982354054, -14826680454, -12219301254, -2840149254, -2208988800 ], + "diffs" : [ -171546, -85146, 1254, 87654, 174054, 260454, 346854, 433254, 519654, 606054, 692454, 778854, 865254, 1254, 1240, 0 ] +}, { + "tz" : "Asia/Thimbu", + "switches" : [ -62135618316, -59006383116, -55850709516, -52695035916, -46383602316, -43227928716, -40072255116, -33760821516, -30605147916, -27449474316, -21138040716, -17982367116, -14826693516, -12219314316, -2208988800 ], + "diffs" : [ -172884, -86484, -84, 86316, 172716, 259116, 345516, 431916, 518316, 604716, 691116, 777516, 863916, -84, 0 ] +}, { + "tz" : "Asia/Thimphu", + "switches" : [ -62135618316, -59006383116, -55850709516, -52695035916, -46383602316, -43227928716, -40072255116, -33760821516, -30605147916, -27449474316, -21138040716, -17982367116, -14826693516, -12219314316, -2208988800 ], + "diffs" : [ -172884, -86484, -84, 86316, 172716, 259116, 345516, 431916, 518316, 604716, 691116, 777516, 863916, -84, 0 ] +}, { + "tz" : "Asia/Tokyo", + "switches" : [ -62135630339, -59006395139, -55850721539, -52695047939, -46383614339, -43227940739, -40072267139, -33760833539, -30605159939, -27449486339, -21138052739, -17982379139, -14826705539, -12219326339, -2587712400 ], + "diffs" : [ -171661, -85261, 1139, 87539, 173939, 260339, 346739, 433139, 519539, 605939, 692339, 778739, 865139, 1139, 0 ] +}, { + "tz" : "Asia/Tomsk", + "switches" : [ -62135617191, -59006381991, -55850708391, -52695034791, -46383601191, -43227927591, -40072253991, -33760820391, -30605146791, -27449473191, -21138039591, -17982365991, -14826692391, -12219313191, -2208988800 ], + "diffs" : [ -177609, -91209, -4809, 81591, 167991, 254391, 340791, 427191, 513591, 599991, 686391, 772791, 859191, -4809, 0 ] +}, { + "tz" : "Asia/Ujung_Pandang", + "switches" : [ -62135625456, -59006390256, -55850716656, -52695043056, -46383609456, -43227935856, -40072262256, -33760828656, -30605155056, -27449481456, -21138047856, -17982374256, -14826700656, -12219321456, -2208988800 ], + "diffs" : [ -172944, -86544, -144, 86256, 172656, 259056, 345456, 431856, 518256, 604656, 691056, 777456, 863856, -144, 0 ] +}, { + "tz" : "Asia/Ulaanbaatar", + "switches" : [ -62135622452, -59006387252, -55850713652, -52695040052, -46383606452, -43227932852, -40072259252, -33760825652, -30605152052, -27449478452, -21138044852, -17982371252, -14826697652, -12219318452, -2208988800 ], + "diffs" : [ -175948, -89548, -3148, 83252, 169652, 256052, 342452, 428852, 515252, 601652, 688052, 774452, 860852, -3148, 0 ] +}, { + "tz" : "Asia/Ulan_Bator", + "switches" : [ -62135622452, -59006387252, -55850713652, -52695040052, -46383606452, -43227932852, -40072259252, -33760825652, -30605152052, -27449478452, -21138044852, -17982371252, -14826697652, -12219318452, -2208988800 ], + "diffs" : [ -175948, -89548, -3148, 83252, 169652, 256052, 342452, 428852, 515252, 601652, 688052, 774452, 860852, -3148, 0 ] +}, { + "tz" : "Asia/Urumqi", + "switches" : [ -62135617820, -59006382620, -55850709020, -52695035420, -46383601820, -43227928220, -40072254620, -33760821020, -30605147420, -27449473820, -21138040220, -17982366620, -14826693020, -12219313820, -2208988800 ], + "diffs" : [ -173380, -86980, -580, 85820, 172220, 258620, 345020, 431420, 517820, 604220, 690620, 777020, 863420, -580, 0 ] +}, { + "tz" : "Asia/Ust-Nera", + "switches" : [ -62135631174, -59006395974, -55850722374, -52695048774, -46383615174, -43227941574, -40072267974, -33760834374, -30605160774, -27449487174, -21138053574, -17982379974, -14826706374, -12219327174, -2208988800 ], + "diffs" : [ -174426, -88026, -1626, 84774, 171174, 257574, 343974, 430374, 516774, 603174, 689574, 775974, 862374, -1626, 0 ] +}, { + "tz" : "Asia/Vientiane", + "switches" : [ -62135620924, -59006385724, -55850712124, -52695038524, -46383604924, -43227931324, -40072257724, -33760824124, -30605150524, -27449476924, -21138043324, -17982369724, -14826696124, -12219316924, -2208988800 ], + "diffs" : [ -173876, -87476, -1076, 85324, 171724, 258124, 344524, 430924, 517324, 603724, 690124, 776524, 862924, -1076, 0 ] +}, { + "tz" : "Asia/Vladivostok", + "switches" : [ -62135628451, -59006393251, -55850719651, -52695046051, -46383612451, -43227938851, -40072265251, -33760831651, -30605158051, -27449484451, -21138050851, -17982377251, -14826703651, -12219324451, -2208988800 ], + "diffs" : [ -177149, -90749, -4349, 82051, 168451, 254851, 341251, 427651, 514051, 600451, 686851, 773251, 859651, -4349, 0 ] +}, { + "tz" : "Asia/Yakutsk", + "switches" : [ -62135627938, -59006392738, -55850719138, -52695045538, -46383611938, -43227938338, -40072264738, -33760831138, -30605157538, -27449483938, -21138050338, -17982376738, -14826703138, -12219323938, -2208988800 ], + "diffs" : [ -174062, -87662, -1262, 85138, 171538, 257938, 344338, 430738, 517138, 603538, 689938, 776338, 862738, -1262, 0 ] +}, { + "tz" : "Asia/Yangon", + "switches" : [ -62135619887, -59006384687, -55850711087, -52695037487, -46383603887, -43227930287, -40072256687, -33760823087, -30605149487, -27449475887, -21138042287, -17982368687, -14826695087, -12219315887, -2208988800 ], + "diffs" : [ -173113, -86713, -313, 86087, 172487, 258887, 345287, 431687, 518087, 604487, 690887, 777287, 863687, -313, 0 ] +}, { + "tz" : "Asia/Yekaterinburg", + "switches" : [ -62135611353, -59006376153, -55850702553, -52695028953, -46383595353, -43227921753, -40072248153, -33760814553, -30605140953, -27449467353, -21138033753, -17982360153, -14826686553, -12219307353, -2208988800 ], + "diffs" : [ -176247, -89847, -3447, 82953, 169353, 255753, 342153, 428553, 514953, 601353, 687753, 774153, 860553, -3447, 0 ] +}, { + "tz" : "Asia/Yerevan", + "switches" : [ -62135607480, -59006372280, -55850698680, -52695025080, -46383591480, -43227917880, -40072244280, -33760810680, -30605137080, -27449463480, -21138029880, -17982356280, -14826682680, -12219303480, -2208988800 ], + "diffs" : [ -176520, -90120, -3720, 82680, 169080, 255480, 341880, 428280, 514680, 601080, 687480, 773880, 860280, -3720, 0 ] +}, { + "tz" : "Atlantic/Azores", + "switches" : [ -62135590640, -59006355440, -55850681840, -52695008240, -46383574640, -43227901040, -40072227440, -33760793840, -30605120240, -27449446640, -21138013040, -17982339440, -14826665840, -12219286640, -2713904240, -2208988800 ], + "diffs" : [ -175360, -88960, -2560, 83840, 170240, 256640, 343040, 429440, 515840, 602240, 688640, 775040, 861440, -2560, -3272, 0 ] +}, { + "tz" : "Atlantic/Bermuda", + "switches" : [ -62135581242, -59006346042, -55850672442, -52694998842, -46383565242, -43227891642, -40072218042, -33760784442, -30605110842, -27449437242, -21138003642, -17982330042, -14826656442, -12219277242, -2208988800 ], + "diffs" : [ -173958, -87558, -1158, 85242, 171642, 258042, 344442, 430842, 517242, 603642, 690042, 776442, 862842, -1158, 0 ] +}, { + "tz" : "Atlantic/Canary", + "switches" : [ -62135593104, -59006357904, -55850684304, -52695010704, -46383577104, -43227903504, -40072229904, -33760796304, -30605122704, -27449449104, -21138015504, -17982341904, -14826668304, -12219289104, -2208988800 ], + "diffs" : [ -176496, -90096, -3696, 82704, 169104, 255504, 341904, 428304, 514704, 601104, 687504, 773904, 860304, -3696, 0 ] +}, { + "tz" : "Atlantic/Cape_Verde", + "switches" : [ -62135591156, -59006355956, -55850682356, -52695008756, -46383575156, -43227901556, -40072227956, -33760794356, -30605120756, -27449447156, -21138013556, -17982339956, -14826666356, -12219287156, -2208988800 ], + "diffs" : [ -174844, -88444, -2044, 84356, 170756, 257156, 343556, 429956, 516356, 602756, 689156, 775556, 861956, -2044, 0 ] +}, { + "tz" : "Atlantic/Faeroe", + "switches" : [ -62135595176, -59006359976, -55850686376, -52695012776, -46383579176, -43227905576, -40072231976, -33760798376, -30605124776, -27449451176, -21138017576, -17982343976, -14826670376, -12219291176, -2208988800 ], + "diffs" : [ -174424, -88024, -1624, 84776, 171176, 257576, 343976, 430376, 516776, 603176, 689576, 775976, 862376, -1624, 0 ] +}, { + "tz" : "Atlantic/Faroe", + "switches" : [ -62135595176, -59006359976, -55850686376, -52695012776, -46383579176, -43227905576, -40072231976, -33760798376, -30605124776, -27449451176, -21138017576, -17982343976, -14826670376, -12219291176, -2208988800 ], + "diffs" : [ -174424, -88024, -1624, 84776, 171176, 257576, 343976, 430376, 516776, 603176, 689576, 775976, 862376, -1624, 0 ] +}, { + "tz" : "Atlantic/Jan_Mayen", + "switches" : [ -62135599380, -59006364180, -55850690580, -52695016980, -46383583380, -43227909780, -40072236180, -33760802580, -30605128980, -27449455380, -21138021780, -17982348180, -14826674580, -12219295380, -2366757780 ], + "diffs" : [ -173820, -87420, -1020, 85380, 171780, 258180, 344580, 430980, 517380, 603780, 690180, 776580, 862980, -1020, 0 ] +}, { + "tz" : "Atlantic/Madeira", + "switches" : [ -62135592744, -59006357544, -55850683944, -52695010344, -46383576744, -43227903144, -40072229544, -33760795944, -30605122344, -27449448744, -21138015144, -17982341544, -14826667944, -12219288744, -2208988800 ], + "diffs" : [ -176856, -90456, -4056, 82344, 168744, 255144, 341544, 427944, 514344, 600744, 687144, 773544, 859944, -4056, 0 ] +}, { + "tz" : "Atlantic/Reykjavik", + "switches" : [ -62135591520, -59006356320, -55850682720, -52695009120, -46383575520, -43227901920, -40072228320, -33760794720, -30605121120, -27449447520, -21138013920, -17982340320, -14826666720, -12219287520, -2208988800 ], + "diffs" : [ -178080, -91680, -5280, 81120, 167520, 253920, 340320, 426720, 513120, 599520, 685920, 772320, 858720, -5280, 0 ] +}, { + "tz" : "Atlantic/South_Georgia", + "switches" : [ -62135588032, -59006352832, -55850679232, -52695005632, -46383572032, -43227898432, -40072224832, -33760791232, -30605117632, -27449444032, -21138010432, -17982336832, -14826663232, -12219284032, -2524512832 ], + "diffs" : [ -174368, -87968, -1568, 84832, 171232, 257632, 344032, 430432, 516832, 603232, 689632, 776032, 862432, -1568, 0 ] +}, { + "tz" : "Atlantic/St_Helena", + "switches" : [ -62135595832, -59006360632, -55850687032, -52695013432, -46383579832, -43227906232, -40072232632, -33760799032, -30605125432, -27449451832, -21138018232, -17982344632, -14826671032, -12219291832, -2208988800 ], + "diffs" : [ -173768, -87368, -968, 85432, 171832, 258232, 344632, 431032, 517432, 603832, 690232, 776632, 863032, -968, 0 ] +}, { + "tz" : "Atlantic/Stanley", + "switches" : [ -62135582916, -59006347716, -55850674116, -52695000516, -46383566916, -43227893316, -40072219716, -33760786116, -30605112516, -27449438916, -21138005316, -17982331716, -14826658116, -12219278916, -2208988800 ], + "diffs" : [ -175884, -89484, -3084, 83316, 169716, 256116, 342516, 428916, 515316, 601716, 688116, 774516, 860916, -3084, 0 ] +}, { + "tz" : "Australia/ACT", + "switches" : [ -62135633092, -59006397892, -55850724292, -52695050692, -46383617092, -43227943492, -40072269892, -33760836292, -30605162692, -27449489092, -21138055492, -17982381892, -14826708292, -12219329092, -2364113092 ], + "diffs" : [ -172508, -86108, 292, 86692, 173092, 259492, 345892, 432292, 518692, 605092, 691492, 777892, 864292, 292, 0 ] +}, { + "tz" : "Australia/Adelaide", + "switches" : [ -62135630060, -59006394860, -55850721260, -52695047660, -46383614060, -43227940460, -40072266860, -33760833260, -30605159660, -27449486060, -21138052460, -17982378860, -14826705260, -12219326060, -2364110060, -2230189200 ], + "diffs" : [ -173740, -87340, -940, 85460, 171860, 258260, 344660, 431060, 517460, 603860, 690260, 776660, 863060, -940, -1800, 0 ] +}, { + "tz" : "Australia/Brisbane", + "switches" : [ -62135633528, -59006398328, -55850724728, -52695051128, -46383617528, -43227943928, -40072270328, -33760836728, -30605163128, -27449489528, -21138055928, -17982382328, -14826708728, -12219329528, -2366791928 ], + "diffs" : [ -172072, -85672, 728, 87128, 173528, 259928, 346328, 432728, 519128, 605528, 691928, 778328, 864728, 728, 0 ] +}, { + "tz" : "Australia/Broken_Hill", + "switches" : [ -62135630748, -59006395548, -55850721948, -52695048348, -46383614748, -43227941148, -40072267548, -33760833948, -30605160348, -27449486748, -21138053148, -17982379548, -14826705948, -12219326748, -2364110748, -2314951200, -2230189200 ], + "diffs" : [ -173052, -86652, -252, 86148, 172548, 258948, 345348, 431748, 518148, 604548, 690948, 777348, 863748, -252, 1800, -1800, 0 ] +}, { + "tz" : "Australia/Canberra", + "switches" : [ -62135633092, -59006397892, -55850724292, -52695050692, -46383617092, -43227943492, -40072269892, -33760836292, -30605162692, -27449489092, -21138055492, -17982381892, -14826708292, -12219329092, -2364113092 ], + "diffs" : [ -172508, -86108, 292, 86692, 173092, 259492, 345892, 432292, 518692, 605092, 691492, 777892, 864292, 292, 0 ] +}, { + "tz" : "Australia/Currie", + "switches" : [ -62135631328, -59006396128, -55850722528, -52695048928, -46383615328, -43227941728, -40072268128, -33760834528, -30605160928, -27449487328, -21138053728, -17982380128, -14826706528, -12219327328, -2345794528 ], + "diffs" : [ -174272, -87872, -1472, 84928, 171328, 257728, 344128, 430528, 516928, 603328, 689728, 776128, 862528, -1472, 0 ] +}, { + "tz" : "Australia/Darwin", + "switches" : [ -62135628200, -59006393000, -55850719400, -52695045800, -46383612200, -43227938600, -40072265000, -33760831400, -30605157800, -27449484200, -21138050600, -17982377000, -14826703400, -12219324200, -2364108200, -2230189200 ], + "diffs" : [ -175600, -89200, -2800, 83600, 170000, 256400, 342800, 429200, 515600, 602000, 688400, 774800, 861200, -2800, -1800, 0 ] +}, { + "tz" : "Australia/Eucla", + "switches" : [ -62135627728, -59006392528, -55850718928, -52695045328, -46383611728, -43227938128, -40072264528, -33760830928, -30605157328, -27449483728, -21138050128, -17982376528, -14826702928, -12219323728, -2337928528 ], + "diffs" : [ -173372, -86972, -572, 85828, 172228, 258628, 345028, 431428, 517828, 604228, 690628, 777028, 863428, -572, 0 ] +}, { + "tz" : "Australia/Hobart", + "switches" : [ -62135632156, -59006396956, -55850723356, -52695049756, -46383616156, -43227942556, -40072268956, -33760835356, -30605161756, -27449488156, -21138054556, -17982380956, -14826707356, -12219328156, -2345795356 ], + "diffs" : [ -173444, -87044, -644, 85756, 172156, 258556, 344956, 431356, 517756, 604156, 690556, 776956, 863356, -644, 0 ] +}, { + "tz" : "Australia/LHI", + "switches" : [ -62135634980, -59006399780, -55850726180, -52695052580, -46383618980, -43227945380, -40072271780, -33760838180, -30605164580, -27449490980, -21138057380, -17982383780, -14826710180, -12219330980, -2364114980, -2208988800 ], + "diffs" : [ -172420, -86020, 380, 86780, 173180, 259580, 345980, 432380, 518780, 605180, 691580, 777980, 864380, 380, -1800, 0 ] +}, { + "tz" : "Australia/Lindeman", + "switches" : [ -62135632556, -59006397356, -55850723756, -52695050156, -46383616556, -43227942956, -40072269356, -33760835756, -30605162156, -27449488556, -21138054956, -17982381356, -14826707756, -12219328556, -2366790956 ], + "diffs" : [ -173044, -86644, -244, 86156, 172556, 258956, 345356, 431756, 518156, 604556, 690956, 777356, 863756, -244, 0 ] +}, { + "tz" : "Australia/Lord_Howe", + "switches" : [ -62135634980, -59006399780, -55850726180, -52695052580, -46383618980, -43227945380, -40072271780, -33760838180, -30605164580, -27449490980, -21138057380, -17982383780, -14826710180, -12219330980, -2364114980, -2208988800 ], + "diffs" : [ -172420, -86020, 380, 86780, 173180, 259580, 345980, 432380, 518780, 605180, 691580, 777980, 864380, 380, -1800, 0 ] +}, { + "tz" : "Australia/Melbourne", + "switches" : [ -62135631592, -59006396392, -55850722792, -52695049192, -46383615592, -43227941992, -40072268392, -33760834792, -30605161192, -27449487592, -21138053992, -17982380392, -14826706792, -12219327592, -2364111592 ], + "diffs" : [ -174008, -87608, -1208, 85192, 171592, 257992, 344392, 430792, 517192, 603592, 689992, 776392, 862792, -1208, 0 ] +}, { + "tz" : "Australia/NSW", + "switches" : [ -62135633092, -59006397892, -55850724292, -52695050692, -46383617092, -43227943492, -40072269892, -33760836292, -30605162692, -27449489092, -21138055492, -17982381892, -14826708292, -12219329092, -2364113092 ], + "diffs" : [ -172508, -86108, 292, 86692, 173092, 259492, 345892, 432292, 518692, 605092, 691492, 777892, 864292, 292, 0 ] +}, { + "tz" : "Australia/North", + "switches" : [ -62135628200, -59006393000, -55850719400, -52695045800, -46383612200, -43227938600, -40072265000, -33760831400, -30605157800, -27449484200, -21138050600, -17982377000, -14826703400, -12219324200, -2364108200, -2230189200 ], + "diffs" : [ -175600, -89200, -2800, 83600, 170000, 256400, 342800, 429200, 515600, 602000, 688400, 774800, 861200, -2800, -1800, 0 ] +}, { + "tz" : "Australia/Perth", + "switches" : [ -62135624604, -59006389404, -55850715804, -52695042204, -46383608604, -43227935004, -40072261404, -33760827804, -30605154204, -27449480604, -21138047004, -17982373404, -14826699804, -12219320604, -2337925404 ], + "diffs" : [ -173796, -87396, -996, 85404, 171804, 258204, 344604, 431004, 517404, 603804, 690204, 776604, 863004, -996, 0 ] +}, { + "tz" : "Australia/Queensland", + "switches" : [ -62135633528, -59006398328, -55850724728, -52695051128, -46383617528, -43227943928, -40072270328, -33760836728, -30605163128, -27449489528, -21138055928, -17982382328, -14826708728, -12219329528, -2366791928 ], + "diffs" : [ -172072, -85672, 728, 87128, 173528, 259928, 346328, 432728, 519128, 605528, 691928, 778328, 864728, 728, 0 ] +}, { + "tz" : "Australia/South", + "switches" : [ -62135630060, -59006394860, -55850721260, -52695047660, -46383614060, -43227940460, -40072266860, -33760833260, -30605159660, -27449486060, -21138052460, -17982378860, -14826705260, -12219326060, -2364110060, -2230189200 ], + "diffs" : [ -173740, -87340, -940, 85460, 171860, 258260, 344660, 431060, 517460, 603860, 690260, 776660, 863060, -940, -1800, 0 ] +}, { + "tz" : "Australia/Sydney", + "switches" : [ -62135633092, -59006397892, -55850724292, -52695050692, -46383617092, -43227943492, -40072269892, -33760836292, -30605162692, -27449489092, -21138055492, -17982381892, -14826708292, -12219329092, -2364113092 ], + "diffs" : [ -172508, -86108, 292, 86692, 173092, 259492, 345892, 432292, 518692, 605092, 691492, 777892, 864292, 292, 0 ] +}, { + "tz" : "Australia/Tasmania", + "switches" : [ -62135632156, -59006396956, -55850723356, -52695049756, -46383616156, -43227942556, -40072268956, -33760835356, -30605161756, -27449488156, -21138054556, -17982380956, -14826707356, -12219328156, -2345795356 ], + "diffs" : [ -173444, -87044, -644, 85756, 172156, 258556, 344956, 431356, 517756, 604156, 690556, 776956, 863356, -644, 0 ] +}, { + "tz" : "Australia/Victoria", + "switches" : [ -62135631592, -59006396392, -55850722792, -52695049192, -46383615592, -43227941992, -40072268392, -33760834792, -30605161192, -27449487592, -21138053992, -17982380392, -14826706792, -12219327592, -2364111592 ], + "diffs" : [ -174008, -87608, -1208, 85192, 171592, 257992, 344392, 430792, 517192, 603592, 689992, 776392, 862792, -1208, 0 ] +}, { + "tz" : "Australia/West", + "switches" : [ -62135624604, -59006389404, -55850715804, -52695042204, -46383608604, -43227935004, -40072261404, -33760827804, -30605154204, -27449480604, -21138047004, -17982373404, -14826699804, -12219320604, -2337925404 ], + "diffs" : [ -173796, -87396, -996, 85404, 171804, 258204, 344604, 431004, 517404, 603804, 690204, 776604, 863004, -996, 0 ] +}, { + "tz" : "Australia/Yancowinna", + "switches" : [ -62135630748, -59006395548, -55850721948, -52695048348, -46383614748, -43227941148, -40072267548, -33760833948, -30605160348, -27449486748, -21138053148, -17982379548, -14826705948, -12219326748, -2364110748, -2314951200, -2230189200 ], + "diffs" : [ -173052, -86652, -252, 86148, 172548, 258948, 345348, 431748, 518148, 604548, 690948, 777348, 863748, -252, 1800, -1800, 0 ] +}, { + "tz" : "Brazil/Acre", + "switches" : [ -62135580528, -59006345328, -55850671728, -52694998128, -46383564528, -43227890928, -40072217328, -33760783728, -30605110128, -27449436528, -21138002928, -17982329328, -14826655728, -12219276528, -2208988800 ], + "diffs" : [ -171072, -84672, 1728, 88128, 174528, 260928, 347328, 433728, 520128, 606528, 692928, 779328, 865728, 1728, 0 ] +}, { + "tz" : "Brazil/DeNoronha", + "switches" : [ -62135589020, -59006353820, -55850680220, -52695006620, -46383573020, -43227899420, -40072225820, -33760792220, -30605118620, -27449445020, -21138011420, -17982337820, -14826664220, -12219285020, -2208988800 ], + "diffs" : [ -173380, -86980, -580, 85820, 172220, 258620, 345020, 431420, 517820, 604220, 690620, 777020, 863420, -580, 0 ] +}, { + "tz" : "Brazil/East", + "switches" : [ -62135585612, -59006350412, -55850676812, -52695003212, -46383569612, -43227896012, -40072222412, -33760788812, -30605115212, -27449441612, -21138008012, -17982334412, -14826660812, -12219281612, -2208988800 ], + "diffs" : [ -173188, -86788, -388, 86012, 172412, 258812, 345212, 431612, 518012, 604412, 690812, 777212, 863612, -388, 0 ] +}, { + "tz" : "Brazil/West", + "switches" : [ -62135582396, -59006347196, -55850673596, -52694999996, -46383566396, -43227892796, -40072219196, -33760785596, -30605111996, -27449438396, -21138004796, -17982331196, -14826657596, -12219278396, -2208988800 ], + "diffs" : [ -172804, -86404, -4, 86396, 172796, 259196, 345596, 431996, 518396, 604796, 691196, 777596, 863996, -4, 0 ] +}, { + "tz" : "CET", + "switches" : [ -62135600400, -59006365200, -55850691600, -52695018000, -46383584400, -43227910800, -40072237200, -33760803600, -30605130000, -27449456400, -21138022800, -17982349200, -14826675600, -12219296400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "CST6CDT", + "switches" : [ -62135575200, -59006340000, -55850666400, -52694992800, -46383559200, -43227885600, -40072212000, -33760778400, -30605104800, -27449431200, -21137997600, -17982324000, -14826650400, -12219271200 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Canada/Atlantic", + "switches" : [ -62135581536, -59006346336, -55850672736, -52694999136, -46383565536, -43227891936, -40072218336, -33760784736, -30605111136, -27449437536, -21138003936, -17982330336, -14826656736, -12219277536, -2208988800 ], + "diffs" : [ -173664, -87264, -864, 85536, 171936, 258336, 344736, 431136, 517536, 603936, 690336, 776736, 863136, -864, 0 ] +}, { + "tz" : "Canada/Central", + "switches" : [ -62135573484, -59006338284, -55850664684, -52694991084, -46383557484, -43227883884, -40072210284, -33760776684, -30605103084, -27449429484, -21137995884, -17982322284, -14826648684, -12219269484, -2602258284 ], + "diffs" : [ -174516, -88116, -1716, 84684, 171084, 257484, 343884, 430284, 516684, 603084, 689484, 775884, 862284, -1716, 0 ] +}, { + "tz" : "Canada/Eastern", + "switches" : [ -62135577748, -59006342548, -55850668948, -52694995348, -46383561748, -43227888148, -40072214548, -33760780948, -30605107348, -27449433748, -21138000148, -17982326548, -14826652948, -12219273748, -2366736148 ], + "diffs" : [ -173852, -87452, -1052, 85348, 171748, 258148, 344548, 430948, 517348, 603748, 690148, 776548, 862948, -1052, 0 ] +}, { + "tz" : "Canada/Mountain", + "switches" : [ -62135569568, -59006334368, -55850660768, -52694987168, -46383553568, -43227879968, -40072206368, -33760772768, -30605099168, -27449425568, -21137991968, -17982318368, -14826644768, -12219265568, -2208988800 ], + "diffs" : [ -174832, -88432, -2032, 84368, 170768, 257168, 343568, 429968, 516368, 602768, 689168, 775568, 861968, -2032, 0 ] +}, { + "tz" : "Canada/Newfoundland", + "switches" : [ -62135584148, -59006348948, -55850675348, -52695001748, -46383568148, -43227894548, -40072220948, -33760787348, -30605113748, -27449440148, -21138006548, -17982332948, -14826659348, -12219280148, -2208988800 ], + "diffs" : [ -172852, -86452, -52, 86348, 172748, 259148, 345548, 431948, 518348, 604748, 691148, 777548, 863948, -52, 0 ] +}, { + "tz" : "Canada/Pacific", + "switches" : [ -62135567252, -59006332052, -55850658452, -52694984852, -46383551252, -43227877652, -40072204052, -33760770452, -30605096852, -27449423252, -21137989652, -17982316052, -14826642452, -12219263252, -2713880852 ], + "diffs" : [ -173548, -87148, -748, 85652, 172052, 258452, 344852, 431252, 517652, 604052, 690452, 776852, 863252, -748, 0 ] +}, { + "tz" : "Canada/Saskatchewan", + "switches" : [ -62135571684, -59006336484, -55850662884, -52694989284, -46383555684, -43227882084, -40072208484, -33760774884, -30605101284, -27449427684, -21137994084, -17982320484, -14826646884, -12219267684, -2208988800 ], + "diffs" : [ -176316, -89916, -3516, 82884, 169284, 255684, 342084, 428484, 514884, 601284, 687684, 774084, 860484, -3516, 0 ] +}, { + "tz" : "Canada/Yukon", + "switches" : [ -62135564388, -59006329188, -55850655588, -52694981988, -46383548388, -43227874788, -40072201188, -33760767588, -30605093988, -27449420388, -21137986788, -17982313188, -14826639588, -12219260388, -2208988800 ], + "diffs" : [ -176412, -90012, -3612, 82788, 169188, 255588, 341988, 428388, 514788, 601188, 687588, 773988, 860388, -3612, 0 ] +}, { + "tz" : "Chile/Continental", + "switches" : [ -62135579834, -59006344634, -55850671034, -52694997434, -46383563834, -43227890234, -40072216634, -33760783034, -30605109434, -27449435834, -21138002234, -17982328634, -14826655034, -12219275834, -2208988800 ], + "diffs" : [ -175366, -88966, -2566, 83834, 170234, 256634, 343034, 429434, 515834, 602234, 688634, 775034, 861434, -2566, 0 ] +}, { + "tz" : "Chile/EasterIsland", + "switches" : [ -62135570552, -59006335352, -55850661752, -52694988152, -46383554552, -43227880952, -40072207352, -33760773752, -30605100152, -27449426552, -21137992952, -17982319352, -14826645752, -12219266552, -2208988800 ], + "diffs" : [ -177448, -91048, -4648, 81752, 168152, 254552, 340952, 427352, 513752, 600152, 686552, 772952, 859352, -4648, 0 ] +}, { + "tz" : "Cuba", + "switches" : [ -62135577032, -59006341832, -55850668232, -52694994632, -46383561032, -43227887432, -40072213832, -33760780232, -30605106632, -27449433032, -21137999432, -17982325832, -14826652232, -12219273032, -2524501832, -2208988800 ], + "diffs" : [ -174568, -88168, -1768, 84632, 171032, 257432, 343832, 430232, 516632, 603032, 689432, 775832, 862232, -1768, -1776, 0 ] +}, { + "tz" : "EET", + "switches" : [ -62135604000, -59006368800, -55850695200, -52695021600, -46383588000, -43227914400, -40072240800, -33760807200, -30605133600, -27449460000, -21138026400, -17982352800, -14826679200, -12219300000 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "EST5EDT", + "switches" : [ -62135578800, -59006343600, -55850670000, -52694996400, -46383562800, -43227889200, -40072215600, -33760782000, -30605108400, -27449434800, -21138001200, -17982327600, -14826654000, -12219274800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Egypt", + "switches" : [ -62135604309, -59006369109, -55850695509, -52695021909, -46383588309, -43227914709, -40072241109, -33760807509, -30605133909, -27449460309, -21138026709, -17982353109, -14826679509, -12219300309, -2208988800 ], + "diffs" : [ -172491, -86091, 309, 86709, 173109, 259509, 345909, 432309, 518709, 605109, 691509, 777909, 864309, 309, 0 ] +}, { + "tz" : "Eire", + "switches" : [ -62135595300, -59006360100, -55850686500, -52695012900, -46383579300, -43227905700, -40072232100, -33760798500, -30605124900, -27449451300, -21138017700, -17982344100, -14826670500, -12219291300, -2821649700, -2208988800 ], + "diffs" : [ -174300, -87900, -1500, 84900, 171300, 257700, 344100, 430500, 516900, 603300, 689700, 776100, 862500, -1500, -1521, 0 ] +}, { + "tz" : "Etc/GMT", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+0", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+1", + "switches" : [ -62135593200, -59006358000, -55850684400, -52695010800, -46383577200, -43227903600, -40072230000, -33760796400, -30605122800, -27449449200, -21138015600, -17982342000, -14826668400, -12219289200 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+10", + "switches" : [ -62135560800, -59006325600, -55850652000, -52694978400, -46383544800, -43227871200, -40072197600, -33760764000, -30605090400, -27449416800, -21137983200, -17982309600, -14826636000, -12219256800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+11", + "switches" : [ -62135557200, -59006322000, -55850648400, -52694974800, -46383541200, -43227867600, -40072194000, -33760760400, -30605086800, -27449413200, -21137979600, -17982306000, -14826632400, -12219253200 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+12", + "switches" : [ -62135553600, -59006318400, -55850644800, -52694971200, -46383537600, -43227864000, -40072190400, -33760756800, -30605083200, -27449409600, -21137976000, -17982302400, -14826628800, -12219249600 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+2", + "switches" : [ -62135589600, -59006354400, -55850680800, -52695007200, -46383573600, -43227900000, -40072226400, -33760792800, -30605119200, -27449445600, -21138012000, -17982338400, -14826664800, -12219285600 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+3", + "switches" : [ -62135586000, -59006350800, -55850677200, -52695003600, -46383570000, -43227896400, -40072222800, -33760789200, -30605115600, -27449442000, -21138008400, -17982334800, -14826661200, -12219282000 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+4", + "switches" : [ -62135582400, -59006347200, -55850673600, -52695000000, -46383566400, -43227892800, -40072219200, -33760785600, -30605112000, -27449438400, -21138004800, -17982331200, -14826657600, -12219278400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+5", + "switches" : [ -62135578800, -59006343600, -55850670000, -52694996400, -46383562800, -43227889200, -40072215600, -33760782000, -30605108400, -27449434800, -21138001200, -17982327600, -14826654000, -12219274800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+6", + "switches" : [ -62135575200, -59006340000, -55850666400, -52694992800, -46383559200, -43227885600, -40072212000, -33760778400, -30605104800, -27449431200, -21137997600, -17982324000, -14826650400, -12219271200 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+7", + "switches" : [ -62135571600, -59006336400, -55850662800, -52694989200, -46383555600, -43227882000, -40072208400, -33760774800, -30605101200, -27449427600, -21137994000, -17982320400, -14826646800, -12219267600 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+8", + "switches" : [ -62135568000, -59006332800, -55850659200, -52694985600, -46383552000, -43227878400, -40072204800, -33760771200, -30605097600, -27449424000, -21137990400, -17982316800, -14826643200, -12219264000 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT+9", + "switches" : [ -62135564400, -59006329200, -55850655600, -52694982000, -46383548400, -43227874800, -40072201200, -33760767600, -30605094000, -27449420400, -21137986800, -17982313200, -14826639600, -12219260400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-0", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-1", + "switches" : [ -62135600400, -59006365200, -55850691600, -52695018000, -46383584400, -43227910800, -40072237200, -33760803600, -30605130000, -27449456400, -21138022800, -17982349200, -14826675600, -12219296400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-10", + "switches" : [ -62135632800, -59006397600, -55850724000, -52695050400, -46383616800, -43227943200, -40072269600, -33760836000, -30605162400, -27449488800, -21138055200, -17982381600, -14826708000, -12219328800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-11", + "switches" : [ -62135636400, -59006401200, -55850727600, -52695054000, -46383620400, -43227946800, -40072273200, -33760839600, -30605166000, -27449492400, -21138058800, -17982385200, -14826711600, -12219332400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-12", + "switches" : [ -62135640000, -59006404800, -55850731200, -52695057600, -46383624000, -43227950400, -40072276800, -33760843200, -30605169600, -27449496000, -21138062400, -17982388800, -14826715200, -12219336000 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-13", + "switches" : [ -62135643600, -59006408400, -55850734800, -52695061200, -46383627600, -43227954000, -40072280400, -33760846800, -30605173200, -27449499600, -21138066000, -17982392400, -14826718800, -12219339600 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-14", + "switches" : [ -62135647200, -59006412000, -55850738400, -52695064800, -46383631200, -43227957600, -40072284000, -33760850400, -30605176800, -27449503200, -21138069600, -17982396000, -14826722400, -12219343200 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-2", + "switches" : [ -62135604000, -59006368800, -55850695200, -52695021600, -46383588000, -43227914400, -40072240800, -33760807200, -30605133600, -27449460000, -21138026400, -17982352800, -14826679200, -12219300000 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-3", + "switches" : [ -62135607600, -59006372400, -55850698800, -52695025200, -46383591600, -43227918000, -40072244400, -33760810800, -30605137200, -27449463600, -21138030000, -17982356400, -14826682800, -12219303600 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-4", + "switches" : [ -62135611200, -59006376000, -55850702400, -52695028800, -46383595200, -43227921600, -40072248000, -33760814400, -30605140800, -27449467200, -21138033600, -17982360000, -14826686400, -12219307200 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-5", + "switches" : [ -62135614800, -59006379600, -55850706000, -52695032400, -46383598800, -43227925200, -40072251600, -33760818000, -30605144400, -27449470800, -21138037200, -17982363600, -14826690000, -12219310800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-6", + "switches" : [ -62135618400, -59006383200, -55850709600, -52695036000, -46383602400, -43227928800, -40072255200, -33760821600, -30605148000, -27449474400, -21138040800, -17982367200, -14826693600, -12219314400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-7", + "switches" : [ -62135622000, -59006386800, -55850713200, -52695039600, -46383606000, -43227932400, -40072258800, -33760825200, -30605151600, -27449478000, -21138044400, -17982370800, -14826697200, -12219318000 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-8", + "switches" : [ -62135625600, -59006390400, -55850716800, -52695043200, -46383609600, -43227936000, -40072262400, -33760828800, -30605155200, -27449481600, -21138048000, -17982374400, -14826700800, -12219321600 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT-9", + "switches" : [ -62135629200, -59006394000, -55850720400, -52695046800, -46383613200, -43227939600, -40072266000, -33760832400, -30605158800, -27449485200, -21138051600, -17982378000, -14826704400, -12219325200 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/GMT0", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/Greenwich", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/UCT", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/UTC", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/Universal", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Etc/Zulu", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Europe/Amsterdam", + "switches" : [ -62135597972, -59006362772, -55850689172, -52695015572, -46383581972, -43227908372, -40072234772, -33760801172, -30605127572, -27449453972, -21138020372, -17982346772, -14826673172, -12219293972, -2208988800 ], + "diffs" : [ -175228, -88828, -2428, 83972, 170372, 256772, 343172, 429572, 515972, 602372, 688772, 775172, 861572, -2428, 0 ] +}, { + "tz" : "Europe/Andorra", + "switches" : [ -62135597164, -59006361964, -55850688364, -52695014764, -46383581164, -43227907564, -40072233964, -33760800364, -30605126764, -27449453164, -21138019564, -17982345964, -14826672364, -12219293164, -2208988800 ], + "diffs" : [ -176036, -89636, -3236, 83164, 169564, 255964, 342364, 428764, 515164, 601564, 687964, 774364, 860764, -3236, 0 ] +}, { + "tz" : "Europe/Astrakhan", + "switches" : [ -62135608332, -59006373132, -55850699532, -52695025932, -46383592332, -43227918732, -40072245132, -33760811532, -30605137932, -27449464332, -21138030732, -17982357132, -14826683532, -12219304332, -2208988800 ], + "diffs" : [ -175668, -89268, -2868, 83532, 169932, 256332, 342732, 429132, 515532, 601932, 688332, 774732, 861132, -2868, 0 ] +}, { + "tz" : "Europe/Athens", + "switches" : [ -62135602492, -59006367292, -55850693692, -52695020092, -46383586492, -43227912892, -40072239292, -33760805692, -30605132092, -27449458492, -21138024892, -17982351292, -14826677692, -12219298492, -2208988800 ], + "diffs" : [ -174308, -87908, -1508, 84892, 171292, 257692, 344092, 430492, 516892, 603292, 689692, 776092, 862492, -1508, 0 ] +}, { + "tz" : "Europe/Belfast", + "switches" : [ -62135596725, -59006361525, -55850687925, -52695014325, -46383580725, -43227907125, -40072233525, -33760799925, -30605126325, -27449452725, -21138019125, -17982345525, -14826671925, -12219292725, -3852662325 ], + "diffs" : [ -172875, -86475, -75, 86325, 172725, 259125, 345525, 431925, 518325, 604725, 691125, 777525, 863925, -75, 0 ] +}, { + "tz" : "Europe/Belgrade", + "switches" : [ -62135601720, -59006366520, -55850692920, -52695019320, -46383585720, -43227912120, -40072238520, -33760804920, -30605131320, -27449457720, -21138024120, -17982350520, -14826676920, -12219297720, -2713915320 ], + "diffs" : [ -171480, -85080, 1320, 87720, 174120, 260520, 346920, 433320, 519720, 606120, 692520, 778920, 865320, 1320, 0 ] +}, { + "tz" : "Europe/Berlin", + "switches" : [ -62135600008, -59006364808, -55850691208, -52695017608, -46383584008, -43227910408, -40072236808, -33760803208, -30605129608, -27449456008, -21138022408, -17982348808, -14826675208, -12219296008, -2422054408 ], + "diffs" : [ -173192, -86792, -392, 86008, 172408, 258808, 345208, 431608, 518008, 604408, 690808, 777208, 863608, -392, 0 ] +}, { + "tz" : "Europe/Bratislava", + "switches" : [ -62135600264, -59006365064, -55850691464, -52695017864, -46383584264, -43227910664, -40072237064, -33760803464, -30605129864, -27449456264, -21138022664, -17982349064, -14826675464, -12219296264, -2469401864 ], + "diffs" : [ -172936, -86536, -136, 86264, 172664, 259064, 345464, 431864, 518264, 604664, 691064, 777464, 863864, -136, 0 ] +}, { + "tz" : "Europe/Brussels", + "switches" : [ -62135597850, -59006362650, -55850689050, -52695015450, -46383581850, -43227908250, -40072234650, -33760801050, -30605127450, -27449453850, -21138020250, -17982346650, -14826673050, -12219293850, -2450995200, -2208988800 ], + "diffs" : [ -175350, -88950, -2550, 83850, 170250, 256650, 343050, 429450, 515850, 602250, 688650, 775050, 861450, -2550, -3600, 0 ] +}, { + "tz" : "Europe/Bucharest", + "switches" : [ -62135603064, -59006367864, -55850694264, -52695020664, -46383587064, -43227913464, -40072239864, -33760806264, -30605132664, -27449459064, -21138025464, -17982351864, -14826678264, -12219299064, -2208988800 ], + "diffs" : [ -173736, -87336, -936, 85464, 171864, 258264, 344664, 431064, 517464, 603864, 690264, 776664, 863064, -936, 0 ] +}, { + "tz" : "Europe/Budapest", + "switches" : [ -62135601380, -59006366180, -55850692580, -52695018980, -46383585380, -43227911780, -40072238180, -33760804580, -30605130980, -27449457380, -21138023780, -17982350180, -14826676580, -12219297380, -2500938980 ], + "diffs" : [ -171820, -85420, 980, 87380, 173780, 260180, 346580, 432980, 519380, 605780, 692180, 778580, 864980, 980, 0 ] +}, { + "tz" : "Europe/Busingen", + "switches" : [ -62135598848, -59006363648, -55850690048, -52695016448, -46383582848, -43227909248, -40072235648, -33760802048, -30605128448, -27449454848, -21138021248, -17982347648, -14826674048, -12219294848, -3675198848, -2385246586 ], + "diffs" : [ -174352, -87952, -1552, 84848, 171248, 257648, 344048, 430448, 516848, 603248, 689648, 776048, 862448, -1552, -1814, 0 ] +}, { + "tz" : "Europe/Chisinau", + "switches" : [ -62135603720, -59006368520, -55850694920, -52695021320, -46383587720, -43227914120, -40072240520, -33760806920, -30605133320, -27449459720, -21138026120, -17982352520, -14826678920, -12219299720, -2840147720, -2208988800 ], + "diffs" : [ -173080, -86680, -280, 86120, 172520, 258920, 345320, 431720, 518120, 604520, 690920, 777320, 863720, -280, -300, 0 ] +}, { + "tz" : "Europe/Copenhagen", + "switches" : [ -62135599820, -59006364620, -55850691020, -52695017420, -46383583820, -43227910220, -40072236620, -33760803020, -30605129420, -27449455820, -21138022220, -17982348620, -14826675020, -12219295820, -2398294220 ], + "diffs" : [ -173380, -86980, -580, 85820, 172220, 258620, 345020, 431420, 517820, 604220, 690620, 777020, 863420, -580, 0 ] +}, { + "tz" : "Europe/Dublin", + "switches" : [ -62135595300, -59006360100, -55850686500, -52695012900, -46383579300, -43227905700, -40072232100, -33760798500, -30605124900, -27449451300, -21138017700, -17982344100, -14826670500, -12219291300, -2821649700, -2208988800 ], + "diffs" : [ -174300, -87900, -1500, 84900, 171300, 257700, 344100, 430500, 516900, 603300, 689700, 776100, 862500, -1500, -1521, 0 ] +}, { + "tz" : "Europe/Gibraltar", + "switches" : [ -62135595516, -59006360316, -55850686716, -52695013116, -46383579516, -43227905916, -40072232316, -33760798716, -30605125116, -27449451516, -21138017916, -17982344316, -14826670716, -12219291516, -2821649916, -2208988800 ], + "diffs" : [ -177684, -91284, -4884, 81516, 167916, 254316, 340716, 427116, 513516, 599916, 686316, 772716, 859116, -4884, -3600, 0 ] +}, { + "tz" : "Europe/Guernsey", + "switches" : [ -62135596725, -59006361525, -55850687925, -52695014325, -46383580725, -43227907125, -40072233525, -33760799925, -30605126325, -27449452725, -21138019125, -17982345525, -14826671925, -12219292725, -3852662325 ], + "diffs" : [ -172875, -86475, -75, 86325, 172725, 259125, 345525, 431925, 518325, 604725, 691125, 777525, 863925, -75, 0 ] +}, { + "tz" : "Europe/Helsinki", + "switches" : [ -62135602789, -59006367589, -55850693989, -52695020389, -46383586789, -43227913189, -40072239589, -33760805989, -30605132389, -27449458789, -21138025189, -17982351589, -14826677989, -12219298789, -2208988800 ], + "diffs" : [ -174011, -87611, -1211, 85189, 171589, 257989, 344389, 430789, 517189, 603589, 689989, 776389, 862789, -1211, 0 ] +}, { + "tz" : "Europe/Isle_of_Man", + "switches" : [ -62135596725, -59006361525, -55850687925, -52695014325, -46383580725, -43227907125, -40072233525, -33760799925, -30605126325, -27449452725, -21138019125, -17982345525, -14826671925, -12219292725, -3852662325 ], + "diffs" : [ -172875, -86475, -75, 86325, 172725, 259125, 345525, 431925, 518325, 604725, 691125, 777525, 863925, -75, 0 ] +}, { + "tz" : "Europe/Istanbul", + "switches" : [ -62135603752, -59006368552, -55850694952, -52695021352, -46383587752, -43227914152, -40072240552, -33760806952, -30605133352, -27449459752, -21138026152, -17982352552, -14826678952, -12219299752, -2840147752, -2208988800 ], + "diffs" : [ -176648, -90248, -3848, 82552, 168952, 255352, 341752, 428152, 514552, 600952, 687352, 773752, 860152, -3848, -3784, 0 ] +}, { + "tz" : "Europe/Jersey", + "switches" : [ -62135596725, -59006361525, -55850687925, -52695014325, -46383580725, -43227907125, -40072233525, -33760799925, -30605126325, -27449452725, -21138019125, -17982345525, -14826671925, -12219292725, -3852662325 ], + "diffs" : [ -172875, -86475, -75, 86325, 172725, 259125, 345525, 431925, 518325, 604725, 691125, 777525, 863925, -75, 0 ] +}, { + "tz" : "Europe/Kaliningrad", + "switches" : [ -62135601720, -59006366520, -55850692920, -52695019320, -46383585720, -43227912120, -40072238520, -33760804920, -30605131320, -27449457720, -21138024120, -17982350520, -14826676920, -12219297720, -2422056120, -2208988800 ], + "diffs" : [ -175080, -88680, -2280, 84120, 170520, 256920, 343320, 429720, 516120, 602520, 688920, 775320, 861720, -2280, -3600, 0 ] +}, { + "tz" : "Europe/Kiev", + "switches" : [ -62135604124, -59006368924, -55850695324, -52695021724, -46383588124, -43227914524, -40072240924, -33760807324, -30605133724, -27449460124, -21138026524, -17982352924, -14826679324, -12219300124, -2208988800 ], + "diffs" : [ -172676, -86276, 124, 86524, 172924, 259324, 345724, 432124, 518524, 604924, 691324, 777724, 864124, 124, 0 ] +}, { + "tz" : "Europe/Kirov", + "switches" : [ -62135608728, -59006373528, -55850699928, -52695026328, -46383592728, -43227919128, -40072245528, -33760811928, -30605138328, -27449464728, -21138031128, -17982357528, -14826683928, -12219304728, -2208988800 ], + "diffs" : [ -171672, -85272, 1128, 87528, 173928, 260328, 346728, 433128, 519528, 605928, 692328, 778728, 865128, 1128, 0 ] +}, { + "tz" : "Europe/Lisbon", + "switches" : [ -62135594595, -59006359395, -55850685795, -52695012195, -46383578595, -43227904995, -40072231395, -33760797795, -30605124195, -27449450595, -21138016995, -17982343395, -14826669795, -12219290595, -2208988800 ], + "diffs" : [ -175005, -88605, -2205, 84195, 170595, 256995, 343395, 429795, 516195, 602595, 688995, 775395, 861795, -2205, 0 ] +}, { + "tz" : "Europe/Ljubljana", + "switches" : [ -62135601720, -59006366520, -55850692920, -52695019320, -46383585720, -43227912120, -40072238520, -33760804920, -30605131320, -27449457720, -21138024120, -17982350520, -14826676920, -12219297720, -2713915320 ], + "diffs" : [ -171480, -85080, 1320, 87720, 174120, 260520, 346920, 433320, 519720, 606120, 692520, 778920, 865320, 1320, 0 ] +}, { + "tz" : "Europe/London", + "switches" : [ -62135596725, -59006361525, -55850687925, -52695014325, -46383580725, -43227907125, -40072233525, -33760799925, -30605126325, -27449452725, -21138019125, -17982345525, -14826671925, -12219292725, -3852662325 ], + "diffs" : [ -172875, -86475, -75, 86325, 172725, 259125, 345525, 431925, 518325, 604725, 691125, 777525, 863925, -75, 0 ] +}, { + "tz" : "Europe/Luxembourg", + "switches" : [ -62135598276, -59006363076, -55850689476, -52695015876, -46383582276, -43227908676, -40072235076, -33760801476, -30605127876, -27449454276, -21138020676, -17982347076, -14826673476, -12219294276, -2208988800 ], + "diffs" : [ -174924, -88524, -2124, 84276, 170676, 257076, 343476, 429876, 516276, 602676, 689076, 775476, 861876, -2124, 0 ] +}, { + "tz" : "Europe/Madrid", + "switches" : [ -62135595916, -59006360716, -55850687116, -52695013516, -46383579916, -43227906316, -40072232716, -33760799116, -30605125516, -27449451916, -21138018316, -17982344716, -14826671116, -12219291916, -2208988800 ], + "diffs" : [ -177284, -90884, -4484, 81916, 168316, 254716, 341116, 427516, 513916, 600316, 686716, 773116, 859516, -4484, 0 ] +}, { + "tz" : "Europe/Malta", + "switches" : [ -62135600284, -59006365084, -55850691484, -52695017884, -46383584284, -43227910684, -40072237084, -33760803484, -30605129884, -27449456284, -21138022684, -17982349084, -14826675484, -12219296284, -2403478684 ], + "diffs" : [ -172916, -86516, -116, 86284, 172684, 259084, 345484, 431884, 518284, 604684, 691084, 777484, 863884, -116, 0 ] +}, { + "tz" : "Europe/Mariehamn", + "switches" : [ -62135602789, -59006367589, -55850693989, -52695020389, -46383586789, -43227913189, -40072239589, -33760805989, -30605132389, -27449458789, -21138025189, -17982351589, -14826677989, -12219298789, -2208988800 ], + "diffs" : [ -174011, -87611, -1211, 85189, 171589, 257989, 344389, 430789, 517189, 603589, 689989, 776389, 862789, -1211, 0 ] +}, { + "tz" : "Europe/Minsk", + "switches" : [ -62135603416, -59006368216, -55850694616, -52695021016, -46383587416, -43227913816, -40072240216, -33760806616, -30605133016, -27449459416, -21138025816, -17982352216, -14826678616, -12219299416, -2840147416, -2208988800 ], + "diffs" : [ -176984, -90584, -4184, 82216, 168616, 255016, 341416, 427816, 514216, 600616, 687016, 773416, 859816, -4184, -4200, 0 ] +}, { + "tz" : "Europe/Monaco", + "switches" : [ -62135598572, -59006363372, -55850689772, -52695016172, -46383582572, -43227908972, -40072235372, -33760801772, -30605128172, -27449454572, -21138020972, -17982347372, -14826673772, -12219294572, -2486680172, -2208988800 ], + "diffs" : [ -174628, -88228, -1828, 84572, 170972, 257372, 343772, 430172, 516572, 602972, 689372, 775772, 862172, -1828, -3039, 0 ] +}, { + "tz" : "Europe/Moscow", + "switches" : [ -62135605817, -59006370617, -55850697017, -52695023417, -46383589817, -43227916217, -40072242617, -33760809017, -30605135417, -27449461817, -21138028217, -17982354617, -14826681017, -12219301817, -2208988800 ], + "diffs" : [ -174583, -88183, -1783, 84617, 171017, 257417, 343817, 430217, 516617, 603017, 689417, 775817, 862217, -1783, 0 ] +}, { + "tz" : "Europe/Nicosia", + "switches" : [ -62135604808, -59006369608, -55850696008, -52695022408, -46383588808, -43227915208, -40072241608, -33760808008, -30605134408, -27449460808, -21138027208, -17982353608, -14826680008, -12219300808, -2208988800 ], + "diffs" : [ -171992, -85592, 808, 87208, 173608, 260008, 346408, 432808, 519208, 605608, 692008, 778408, 864808, 808, 0 ] +}, { + "tz" : "Europe/Oslo", + "switches" : [ -62135599380, -59006364180, -55850690580, -52695016980, -46383583380, -43227909780, -40072236180, -33760802580, -30605128980, -27449455380, -21138021780, -17982348180, -14826674580, -12219295380, -2366757780 ], + "diffs" : [ -173820, -87420, -1020, 85380, 171780, 258180, 344580, 430980, 517380, 603780, 690180, 776580, 862980, -1020, 0 ] +}, { + "tz" : "Europe/Paris", + "switches" : [ -62135597361, -59006362161, -55850688561, -52695014961, -46383581361, -43227907761, -40072234161, -33760800561, -30605126961, -27449453361, -21138019761, -17982346161, -14826672561, -12219293361, -2208988800 ], + "diffs" : [ -175839, -89439, -3039, 83361, 169761, 256161, 342561, 428961, 515361, 601761, 688161, 774561, 860961, -3039, 0 ] +}, { + "tz" : "Europe/Podgorica", + "switches" : [ -62135601720, -59006366520, -55850692920, -52695019320, -46383585720, -43227912120, -40072238520, -33760804920, -30605131320, -27449457720, -21138024120, -17982350520, -14826676920, -12219297720, -2713915320 ], + "diffs" : [ -171480, -85080, 1320, 87720, 174120, 260520, 346920, 433320, 519720, 606120, 692520, 778920, 865320, 1320, 0 ] +}, { + "tz" : "Europe/Prague", + "switches" : [ -62135600264, -59006365064, -55850691464, -52695017864, -46383584264, -43227910664, -40072237064, -33760803464, -30605129864, -27449456264, -21138022664, -17982349064, -14826675464, -12219296264, -2469401864 ], + "diffs" : [ -172936, -86536, -136, 86264, 172664, 259064, 345464, 431864, 518264, 604664, 691064, 777464, 863864, -136, 0 ] +}, { + "tz" : "Europe/Riga", + "switches" : [ -62135602594, -59006367394, -55850693794, -52695020194, -46383586594, -43227912994, -40072239394, -33760805794, -30605132194, -27449458594, -21138024994, -17982351394, -14826677794, -12219298594, -2208988800 ], + "diffs" : [ -174206, -87806, -1406, 84994, 171394, 257794, 344194, 430594, 516994, 603394, 689794, 776194, 862594, -1406, 0 ] +}, { + "tz" : "Europe/Rome", + "switches" : [ -62135599796, -59006364596, -55850690996, -52695017396, -46383583796, -43227910196, -40072236596, -33760802996, -30605129396, -27449455796, -21138022196, -17982348596, -14826674996, -12219295796, -2403565200 ], + "diffs" : [ -173404, -87004, -604, 85796, 172196, 258596, 344996, 431396, 517796, 604196, 690596, 776996, 863396, -604, 0 ] +}, { + "tz" : "Europe/Samara", + "switches" : [ -62135608820, -59006373620, -55850700020, -52695026420, -46383592820, -43227919220, -40072245620, -33760812020, -30605138420, -27449464820, -21138031220, -17982357620, -14826684020, -12219304820, -2208988800 ], + "diffs" : [ -175180, -88780, -2380, 84020, 170420, 256820, 343220, 429620, 516020, 602420, 688820, 775220, 861620, -2380, 0 ] +}, { + "tz" : "Europe/San_Marino", + "switches" : [ -62135599796, -59006364596, -55850690996, -52695017396, -46383583796, -43227910196, -40072236596, -33760802996, -30605129396, -27449455796, -21138022196, -17982348596, -14826674996, -12219295796, -2403565200 ], + "diffs" : [ -173404, -87004, -604, 85796, 172196, 258596, 344996, 431396, 517796, 604196, 690596, 776996, 863396, -604, 0 ] +}, { + "tz" : "Europe/Sarajevo", + "switches" : [ -62135601720, -59006366520, -55850692920, -52695019320, -46383585720, -43227912120, -40072238520, -33760804920, -30605131320, -27449457720, -21138024120, -17982350520, -14826676920, -12219297720, -2713915320 ], + "diffs" : [ -171480, -85080, 1320, 87720, 174120, 260520, 346920, 433320, 519720, 606120, 692520, 778920, 865320, 1320, 0 ] +}, { + "tz" : "Europe/Saratov", + "switches" : [ -62135607858, -59006372658, -55850699058, -52695025458, -46383591858, -43227918258, -40072244658, -33760811058, -30605137458, -27449463858, -21138030258, -17982356658, -14826683058, -12219303858, -2208988800 ], + "diffs" : [ -176142, -89742, -3342, 83058, 169458, 255858, 342258, 428658, 515058, 601458, 687858, 774258, 860658, -3342, 0 ] +}, { + "tz" : "Europe/Simferopol", + "switches" : [ -62135604984, -59006369784, -55850696184, -52695022584, -46383588984, -43227915384, -40072241784, -33760808184, -30605134584, -27449460984, -21138027384, -17982353784, -14826680184, -12219300984, -2840148984, -2208988800 ], + "diffs" : [ -175416, -89016, -2616, 83784, 170184, 256584, 342984, 429384, 515784, 602184, 688584, 774984, 861384, -2616, -2640, 0 ] +}, { + "tz" : "Europe/Skopje", + "switches" : [ -62135601720, -59006366520, -55850692920, -52695019320, -46383585720, -43227912120, -40072238520, -33760804920, -30605131320, -27449457720, -21138024120, -17982350520, -14826676920, -12219297720, -2713915320 ], + "diffs" : [ -171480, -85080, 1320, 87720, 174120, 260520, 346920, 433320, 519720, 606120, 692520, 778920, 865320, 1320, 0 ] +}, { + "tz" : "Europe/Sofia", + "switches" : [ -62135602396, -59006367196, -55850693596, -52695019996, -46383586396, -43227912796, -40072239196, -33760805596, -30605131996, -27449458396, -21138024796, -17982351196, -14826677596, -12219298396, -2840146396, -2369527016 ], + "diffs" : [ -174404, -88004, -1604, 84796, 171196, 257596, 343996, 430396, 516796, 603196, 689596, 775996, 862396, -1604, -184, 0 ] +}, { + "tz" : "Europe/Stockholm", + "switches" : [ -62135601132, -59006365932, -55850692332, -52695018732, -46383585132, -43227911532, -40072237932, -33760804332, -30605130732, -27449457132, -21138023532, -17982349932, -14826676332, -12219297132, -2871681132, -2208992414 ], + "diffs" : [ -172068, -85668, 732, 87132, 173532, 259932, 346332, 432732, 519132, 605532, 691932, 778332, 864732, 732, 14, 0 ] +}, { + "tz" : "Europe/Tallinn", + "switches" : [ -62135602740, -59006367540, -55850693940, -52695020340, -46383586740, -43227913140, -40072239540, -33760805940, -30605132340, -27449458740, -21138025140, -17982351540, -14826677940, -12219298740, -2208988800 ], + "diffs" : [ -174060, -87660, -1260, 85140, 171540, 257940, 344340, 430740, 517140, 603540, 689940, 776340, 862740, -1260, 0 ] +}, { + "tz" : "Europe/Tirane", + "switches" : [ -62135601560, -59006366360, -55850692760, -52695019160, -46383585560, -43227911960, -40072238360, -33760804760, -30605131160, -27449457560, -21138023960, -17982350360, -14826676760, -12219297560, -2208988800 ], + "diffs" : [ -171640, -85240, 1160, 87560, 173960, 260360, 346760, 433160, 519560, 605960, 692360, 778760, 865160, 1160, 0 ] +}, { + "tz" : "Europe/Tiraspol", + "switches" : [ -62135603720, -59006368520, -55850694920, -52695021320, -46383587720, -43227914120, -40072240520, -33760806920, -30605133320, -27449459720, -21138026120, -17982352520, -14826678920, -12219299720, -2840147720, -2208988800 ], + "diffs" : [ -173080, -86680, -280, 86120, 172520, 258920, 345320, 431720, 518120, 604520, 690920, 777320, 863720, -280, -300, 0 ] +}, { + "tz" : "Europe/Ulyanovsk", + "switches" : [ -62135608416, -59006373216, -55850699616, -52695026016, -46383592416, -43227918816, -40072245216, -33760811616, -30605138016, -27449464416, -21138030816, -17982357216, -14826683616, -12219304416, -2208988800 ], + "diffs" : [ -175584, -89184, -2784, 83616, 170016, 256416, 342816, 429216, 515616, 602016, 688416, 774816, 861216, -2784, 0 ] +}, { + "tz" : "Europe/Uzhgorod", + "switches" : [ -62135602152, -59006366952, -55850693352, -52695019752, -46383586152, -43227912552, -40072238952, -33760805352, -30605131752, -27449458152, -21138024552, -17982350952, -14826677352, -12219298152, -2500939752, -2208988800 ], + "diffs" : [ -174648, -88248, -1848, 84552, 170952, 257352, 343752, 430152, 516552, 602952, 689352, 775752, 862152, -1848, -3600, 0 ] +}, { + "tz" : "Europe/Vaduz", + "switches" : [ -62135598848, -59006363648, -55850690048, -52695016448, -46383582848, -43227909248, -40072235648, -33760802048, -30605128448, -27449454848, -21138021248, -17982347648, -14826674048, -12219294848, -3675198848, -2385246586 ], + "diffs" : [ -174352, -87952, -1552, 84848, 171248, 257648, 344048, 430448, 516848, 603248, 689648, 776048, 862448, -1552, -1814, 0 ] +}, { + "tz" : "Europe/Vatican", + "switches" : [ -62135599796, -59006364596, -55850690996, -52695017396, -46383583796, -43227910196, -40072236596, -33760802996, -30605129396, -27449455796, -21138022196, -17982348596, -14826674996, -12219295796, -2403565200 ], + "diffs" : [ -173404, -87004, -604, 85796, 172196, 258596, 344996, 431396, 517796, 604196, 690596, 776996, 863396, -604, 0 ] +}, { + "tz" : "Europe/Vienna", + "switches" : [ -62135600721, -59006365521, -55850691921, -52695018321, -46383584721, -43227911121, -40072237521, -33760803921, -30605130321, -27449456721, -21138023121, -17982349521, -14826675921, -12219296721, -2422055121 ], + "diffs" : [ -172479, -86079, 321, 86721, 173121, 259521, 345921, 432321, 518721, 605121, 691521, 777921, 864321, 321, 0 ] +}, { + "tz" : "Europe/Vilnius", + "switches" : [ -62135602876, -59006367676, -55850694076, -52695020476, -46383586876, -43227913276, -40072239676, -33760806076, -30605132476, -27449458876, -21138025276, -17982351676, -14826678076, -12219298876, -2840146876, -2208988800 ], + "diffs" : [ -173924, -87524, -1124, 85276, 171676, 258076, 344476, 430876, 517276, 603676, 690076, 776476, 862876, -1124, -2160, 0 ] +}, { + "tz" : "Europe/Volgograd", + "switches" : [ -62135607460, -59006372260, -55850698660, -52695025060, -46383591460, -43227917860, -40072244260, -33760810660, -30605137060, -27449463460, -21138029860, -17982356260, -14826682660, -12219303460, -2208988800 ], + "diffs" : [ -176540, -90140, -3740, 82660, 169060, 255460, 341860, 428260, 514660, 601060, 687460, 773860, 860260, -3740, 0 ] +}, { + "tz" : "Europe/Warsaw", + "switches" : [ -62135601840, -59006366640, -55850693040, -52695019440, -46383585840, -43227912240, -40072238640, -33760805040, -30605131440, -27449457840, -21138024240, -17982350640, -14826677040, -12219297840, -2208988800 ], + "diffs" : [ -171360, -84960, 1440, 87840, 174240, 260640, 347040, 433440, 519840, 606240, 692640, 779040, 865440, 1440, 0 ] +}, { + "tz" : "Europe/Zagreb", + "switches" : [ -62135601720, -59006366520, -55850692920, -52695019320, -46383585720, -43227912120, -40072238520, -33760804920, -30605131320, -27449457720, -21138024120, -17982350520, -14826676920, -12219297720, -2713915320 ], + "diffs" : [ -171480, -85080, 1320, 87720, 174120, 260520, 346920, 433320, 519720, 606120, 692520, 778920, 865320, 1320, 0 ] +}, { + "tz" : "Europe/Zaporozhye", + "switches" : [ -62135605240, -59006370040, -55850696440, -52695022840, -46383589240, -43227915640, -40072242040, -33760808440, -30605134840, -27449461240, -21138027640, -17982354040, -14826680440, -12219301240, -2840149240, -2208988800 ], + "diffs" : [ -171560, -85160, 1240, 87640, 174040, 260440, 346840, 433240, 519640, 606040, 692440, 778840, 865240, 1240, 1200, 0 ] +}, { + "tz" : "Europe/Zurich", + "switches" : [ -62135598848, -59006363648, -55850690048, -52695016448, -46383582848, -43227909248, -40072235648, -33760802048, -30605128448, -27449454848, -21138021248, -17982347648, -14826674048, -12219294848, -3675198848, -2385246586 ], + "diffs" : [ -174352, -87952, -1552, 84848, 171248, 257648, 344048, 430448, 516848, 603248, 689648, 776048, 862448, -1552, -1814, 0 ] +}, { + "tz" : "GB", + "switches" : [ -62135596725, -59006361525, -55850687925, -52695014325, -46383580725, -43227907125, -40072233525, -33760799925, -30605126325, -27449452725, -21138019125, -17982345525, -14826671925, -12219292725, -3852662325 ], + "diffs" : [ -172875, -86475, -75, 86325, 172725, 259125, 345525, 431925, 518325, 604725, 691125, 777525, 863925, -75, 0 ] +}, { + "tz" : "GB-Eire", + "switches" : [ -62135596725, -59006361525, -55850687925, -52695014325, -46383580725, -43227907125, -40072233525, -33760799925, -30605126325, -27449452725, -21138019125, -17982345525, -14826671925, -12219292725, -3852662325 ], + "diffs" : [ -172875, -86475, -75, 86325, 172725, 259125, 345525, 431925, 518325, 604725, 691125, 777525, 863925, -75, 0 ] +}, { + "tz" : "GMT", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "GMT0", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Greenwich", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Hongkong", + "switches" : [ -62135624202, -59006389002, -55850715402, -52695041802, -46383608202, -43227934602, -40072261002, -33760827402, -30605153802, -27449480202, -21138046602, -17982373002, -14826699402, -12219320202, -2208988800 ], + "diffs" : [ -174198, -87798, -1398, 85002, 171402, 257802, 344202, 430602, 517002, 603402, 689802, 776202, 862602, -1398, 0 ] +}, { + "tz" : "Iceland", + "switches" : [ -62135591520, -59006356320, -55850682720, -52695009120, -46383575520, -43227901920, -40072228320, -33760794720, -30605121120, -27449447520, -21138013920, -17982340320, -14826666720, -12219287520, -2208988800 ], + "diffs" : [ -178080, -91680, -5280, 81120, 167520, 253920, 340320, 426720, 513120, 599520, 685920, 772320, 858720, -5280, 0 ] +}, { + "tz" : "Indian/Antananarivo", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Indian/Chagos", + "switches" : [ -62135614180, -59006378980, -55850705380, -52695031780, -46383598180, -43227924580, -40072250980, -33760817380, -30605143780, -27449470180, -21138036580, -17982362980, -14826689380, -12219310180, -2208988800 ], + "diffs" : [ -177020, -90620, -4220, 82180, 168580, 254980, 341380, 427780, 514180, 600580, 686980, 773380, 859780, -4220, 0 ] +}, { + "tz" : "Indian/Christmas", + "switches" : [ -62135622172, -59006386972, -55850713372, -52695039772, -46383606172, -43227932572, -40072258972, -33760825372, -30605151772, -27449478172, -21138044572, -17982370972, -14826697372, -12219318172, -2364102172 ], + "diffs" : [ -172628, -86228, 172, 86572, 172972, 259372, 345772, 432172, 518572, 604972, 691372, 777772, 864172, 172, 0 ] +}, { + "tz" : "Indian/Cocos", + "switches" : [ -62135620060, -59006384860, -55850711260, -52695037660, -46383604060, -43227930460, -40072256860, -33760823260, -30605149660, -27449476060, -21138042460, -17982368860, -14826695260, -12219316060, -2209012060 ], + "diffs" : [ -172940, -86540, -140, 86260, 172660, 259060, 345460, 431860, 518260, 604660, 691060, 777460, 863860, -140, 0 ] +}, { + "tz" : "Indian/Comoro", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Indian/Kerguelen", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800, -2208988800 ], + "diffs" : [ -190800, -104400, -18000, 68400, 154800, 241200, 327600, 414000, 500400, 586800, 673200, 759600, 846000, -18000, 0 ] +}, { + "tz" : "Indian/Mahe", + "switches" : [ -62135610108, -59006374908, -55850701308, -52695027708, -46383594108, -43227920508, -40072246908, -33760813308, -30605139708, -27449466108, -21138032508, -17982358908, -14826685308, -12219306108, -2208988800 ], + "diffs" : [ -173892, -87492, -1092, 85308, 171708, 258108, 344508, 430908, 517308, 603708, 690108, 776508, 862908, -1092, 0 ] +}, { + "tz" : "Indian/Maldives", + "switches" : [ -62135614440, -59006379240, -55850705640, -52695032040, -46383598440, -43227924840, -40072251240, -33760817640, -30605144040, -27449470440, -21138036840, -17982363240, -14826689640, -12219310440, -2208988800 ], + "diffs" : [ -173160, -86760, -360, 86040, 172440, 258840, 345240, 431640, 518040, 604440, 690840, 777240, 863640, -360, 0 ] +}, { + "tz" : "Indian/Mauritius", + "switches" : [ -62135610600, -59006375400, -55850701800, -52695028200, -46383594600, -43227921000, -40072247400, -33760813800, -30605140200, -27449466600, -21138033000, -17982359400, -14826685800, -12219306600, -2208988800 ], + "diffs" : [ -173400, -87000, -600, 85800, 172200, 258600, 345000, 431400, 517800, 604200, 690600, 777000, 863400, -600, 0 ] +}, { + "tz" : "Indian/Mayotte", + "switches" : [ -62135605636, -59006370436, -55850696836, -52695023236, -46383589636, -43227916036, -40072242436, -33760808836, -30605135236, -27449461636, -21138028036, -17982354436, -14826680836, -12219301636, -2208988800 ], + "diffs" : [ -174764, -88364, -1964, 84436, 170836, 257236, 343636, 430036, 516436, 602836, 689236, 775636, 862036, -1964, 0 ] +}, { + "tz" : "Indian/Reunion", + "switches" : [ -62135610112, -59006374912, -55850701312, -52695027712, -46383594112, -43227920512, -40072246912, -33760813312, -30605139712, -27449466112, -21138032512, -17982358912, -14826685312, -12219306112, -2208988800 ], + "diffs" : [ -173888, -87488, -1088, 85312, 171712, 258112, 344512, 430912, 517312, 603712, 690112, 776512, 862912, -1088, 0 ] +}, { + "tz" : "Israel", + "switches" : [ -62135605254, -59006370054, -55850696454, -52695022854, -46383589254, -43227915654, -40072242054, -33760808454, -30605134854, -27449461254, -21138027654, -17982354054, -14826680454, -12219301254, -2840149254, -2208988800 ], + "diffs" : [ -171546, -85146, 1254, 87654, 174054, 260454, 346854, 433254, 519654, 606054, 692454, 778854, 865254, 1254, 1240, 0 ] +}, { + "tz" : "Jamaica", + "switches" : [ -62135578370, -59006343170, -55850669570, -52694995970, -46383562370, -43227888770, -40072215170, -33760781570, -30605107970, -27449434370, -21138000770, -17982327170, -14826653570, -12219274370, -2208988800 ], + "diffs" : [ -173230, -86830, -430, 85970, 172370, 258770, 345170, 431570, 517970, 604370, 690770, 777170, 863570, -430, 0 ] +}, { + "tz" : "Japan", + "switches" : [ -62135630339, -59006395139, -55850721539, -52695047939, -46383614339, -43227940739, -40072267139, -33760833539, -30605159939, -27449486339, -21138052739, -17982379139, -14826705539, -12219326339, -2587712400 ], + "diffs" : [ -171661, -85261, 1139, 87539, 173939, 260339, 346739, 433139, 519539, 605939, 692339, 778739, 865139, 1139, 0 ] +}, { + "tz" : "Kwajalein", + "switches" : [ -62135636960, -59006401760, -55850728160, -52695054560, -46383620960, -43227947360, -40072273760, -33760840160, -30605166560, -27449492960, -21138059360, -17982385760, -14826712160, -12219332960, -2208988800 ], + "diffs" : [ -175840, -89440, -3040, 83360, 169760, 256160, 342560, 428960, 515360, 601760, 688160, 774560, 860960, -3040, 0 ] +}, { + "tz" : "Libya", + "switches" : [ -62135599964, -59006364764, -55850691164, -52695017564, -46383583964, -43227910364, -40072236764, -33760803164, -30605129564, -27449455964, -21138022364, -17982348764, -14826675164, -12219295964, -2208988800 ], + "diffs" : [ -176836, -90436, -4036, 82364, 168764, 255164, 341564, 427964, 514364, 600764, 687164, 773564, 859964, -4036, 0 ] +}, { + "tz" : "MET", + "switches" : [ -62135600400, -59006365200, -55850691600, -52695018000, -46383584400, -43227910800, -40072237200, -33760803600, -30605130000, -27449456400, -21138022800, -17982349200, -14826675600, -12219296400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "MST7MDT", + "switches" : [ -62135571600, -59006336400, -55850662800, -52694989200, -46383555600, -43227882000, -40072208400, -33760774800, -30605101200, -27449427600, -21137994000, -17982320400, -14826646800, -12219267600 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Mexico/BajaNorte", + "switches" : [ -62135568716, -59006333516, -55850659916, -52694986316, -46383552716, -43227879116, -40072205516, -33760771916, -30605098316, -27449424716, -21137991116, -17982317516, -14826643916, -12219264716, -2208988800 ], + "diffs" : [ -172084, -85684, 716, 87116, 173516, 259916, 346316, 432716, 519116, 605516, 691916, 778316, 864716, 716, 0 ] +}, { + "tz" : "Mexico/BajaSur", + "switches" : [ -62135571260, -59006336060, -55850662460, -52694988860, -46383555260, -43227881660, -40072208060, -33760774460, -30605100860, -27449427260, -21137993660, -17982320060, -14826646460, -12219267260, -2208988800 ], + "diffs" : [ -173140, -86740, -340, 86060, 172460, 258860, 345260, 431660, 518060, 604460, 690860, 777260, 863660, -340, 0 ] +}, { + "tz" : "Mexico/General", + "switches" : [ -62135573004, -59006337804, -55850664204, -52694990604, -46383557004, -43227883404, -40072209804, -33760776204, -30605102604, -27449429004, -21137995404, -17982321804, -14826648204, -12219269004, -2208988800 ], + "diffs" : [ -174996, -88596, -2196, 84204, 170604, 257004, 343404, 429804, 516204, 602604, 689004, 775404, 861804, -2196, 0 ] +}, { + "tz" : "NZ", + "switches" : [ -62135638744, -59006403544, -55850729944, -52695056344, -46383622744, -43227949144, -40072275544, -33760841944, -30605168344, -27449494744, -21138061144, -17982387544, -14826713944, -12219334744, -3192435544, -2208988800 ], + "diffs" : [ -174056, -87656, -1256, 85144, 171544, 257944, 344344, 430744, 517144, 603544, 689944, 776344, 862744, -1256, -1800, 0 ] +}, { + "tz" : "NZ-CHAT", + "switches" : [ -62135640828, -59006405628, -55850732028, -52695058428, -46383624828, -43227951228, -40072277628, -33760844028, -30605170428, -27449496828, -21138063228, -17982389628, -14826716028, -12219336828, -3192437628, -2208988800 ], + "diffs" : [ -174672, -88272, -1872, 84528, 170928, 257328, 343728, 430128, 516528, 602928, 689328, 775728, 862128, -1872, -1800, 0 ] +}, { + "tz" : "Navajo", + "switches" : [ -62135571604, -59006336404, -55850662804, -52694989204, -46383555604, -43227882004, -40072208404, -33760774804, -30605101204, -27449427604, -21137994004, -17982320404, -14826646804, -12219267604, -2717643600 ], + "diffs" : [ -172796, -86396, 4, 86404, 172804, 259204, 345604, 432004, 518404, 604804, 691204, 777604, 864004, 4, 0 ] +}, { + "tz" : "PRC", + "switches" : [ -62135625943, -59006390743, -55850717143, -52695043543, -46383609943, -43227936343, -40072262743, -33760829143, -30605155543, -27449481943, -21138048343, -17982374743, -14826701143, -12219321943, -2208988800 ], + "diffs" : [ -172457, -86057, 343, 86743, 173143, 259543, 345943, 432343, 518743, 605143, 691543, 777943, 864343, 343, 0 ] +}, { + "tz" : "PST8PDT", + "switches" : [ -62135568000, -59006332800, -55850659200, -52694985600, -46383552000, -43227878400, -40072204800, -33760771200, -30605097600, -27449424000, -21137990400, -17982316800, -14826643200, -12219264000 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Pacific/Apia", + "switches" : [ -62135641984, -59006406784, -55850733184, -52695059584, -46383625984, -43227952384, -40072278784, -33760845184, -30605171584, -27449497984, -21138064384, -17982390784, -14826717184, -12219337984, -2445424384, -2208988800 ], + "diffs" : [ -174416, -88016, -1616, 84784, 171184, 257584, 343984, 430384, 516784, 603184, 689584, 775984, 862384, -1616, -88016, 0 ] +}, { + "tz" : "Pacific/Auckland", + "switches" : [ -62135638744, -59006403544, -55850729944, -52695056344, -46383622744, -43227949144, -40072275544, -33760841944, -30605168344, -27449494744, -21138061144, -17982387544, -14826713944, -12219334744, -3192435544, -2208988800 ], + "diffs" : [ -174056, -87656, -1256, 85144, 171544, 257944, 344344, 430744, 517144, 603544, 689944, 776344, 862744, -1256, -1800, 0 ] +}, { + "tz" : "Pacific/Bougainville", + "switches" : [ -62135634136, -59006398936, -55850725336, -52695051736, -46383618136, -43227944536, -40072270936, -33760837336, -30605163736, -27449490136, -21138056536, -17982382936, -14826709336, -12219330136, -2840178136, -2366790512, -2208988800 ], + "diffs" : [ -175064, -88664, -2264, 84136, 170536, 256936, 343336, 429736, 516136, 602536, 688936, 775336, 861736, -2264, -4288, -3600, 0 ] +}, { + "tz" : "Pacific/Chatham", + "switches" : [ -62135640828, -59006405628, -55850732028, -52695058428, -46383624828, -43227951228, -40072277628, -33760844028, -30605170428, -27449496828, -21138063228, -17982389628, -14826716028, -12219336828, -3192437628, -2208988800 ], + "diffs" : [ -174672, -88272, -1872, 84528, 170928, 257328, 343728, 430128, 516528, 602928, 689328, 775728, 862128, -1872, -1800, 0 ] +}, { + "tz" : "Pacific/Chuuk", + "switches" : [ -62135546828, -59006311628, -55850638028, -52694964428, -46383530828, -43227857228, -40072183628, -33760750028, -30605076428, -27449402828, -21137969228, -17982295628, -14826622028, -12219242828, -3944628428, -2208988800 ], + "diffs" : [ -258772, -172372, -85972, 428, 86828, 173228, 259628, 346028, 432428, 518828, 605228, 691628, 778028, -85972, 428, 0 ] +}, { + "tz" : "Pacific/Easter", + "switches" : [ -62135570552, -59006335352, -55850661752, -52694988152, -46383554552, -43227880952, -40072207352, -33760773752, -30605100152, -27449426552, -21137992952, -17982319352, -14826645752, -12219266552, -2208988800 ], + "diffs" : [ -177448, -91048, -4648, 81752, 168152, 254552, 340952, 427352, 513752, 600152, 686552, 772952, 859352, -4648, 0 ] +}, { + "tz" : "Pacific/Efate", + "switches" : [ -62135637196, -59006401996, -55850728396, -52695054796, -46383621196, -43227947596, -40072273996, -33760840396, -30605166796, -27449493196, -21138059596, -17982385996, -14826712396, -12219333196, -2208988800 ], + "diffs" : [ -172004, -85604, 796, 87196, 173596, 259996, 346396, 432796, 519196, 605596, 691996, 778396, 864796, 796, 0 ] +}, { + "tz" : "Pacific/Enderbury", + "switches" : [ -62135555740, -59006320540, -55850646940, -52694973340, -46383539740, -43227866140, -40072192540, -33760758940, -30605085340, -27449411740, -21137978140, -17982304540, -14826630940, -12219251740, -2208988800 ], + "diffs" : [ -260660, -174260, -87860, -1460, 84940, 171340, 257740, 344140, 430540, 516940, 603340, 689740, 776140, -87860, 0 ] +}, { + "tz" : "Pacific/Fakaofo", + "switches" : [ -62135555704, -59006320504, -55850646904, -52694973304, -46383539704, -43227866104, -40072192504, -33760758904, -30605085304, -27449411704, -21137978104, -17982304504, -14826630904, -12219251704, -2208988800 ], + "diffs" : [ -260696, -174296, -87896, -1496, 84904, 171304, 257704, 344104, 430504, 516904, 603304, 689704, 776104, -87896, 0 ] +}, { + "tz" : "Pacific/Fiji", + "switches" : [ -62135639744, -59006404544, -55850730944, -52695057344, -46383623744, -43227950144, -40072276544, -33760842944, -30605169344, -27449495744, -21138062144, -17982388544, -14826714944, -12219335744, -2208988800 ], + "diffs" : [ -173056, -86656, -256, 86144, 172544, 258944, 345344, 431744, 518144, 604544, 690944, 777344, 863744, -256, 0 ] +}, { + "tz" : "Pacific/Funafuti", + "switches" : [ -62135639812, -59006404612, -55850731012, -52695057412, -46383623812, -43227950212, -40072276612, -33760843012, -30605169412, -27449495812, -21138062212, -17982388612, -14826715012, -12219335812, -2208988800 ], + "diffs" : [ -172988, -86588, -188, 86212, 172612, 259012, 345412, 431812, 518212, 604612, 691012, 777412, 863812, -188, 0 ] +}, { + "tz" : "Pacific/Galapagos", + "switches" : [ -62135575296, -59006340096, -55850666496, -52694992896, -46383559296, -43227885696, -40072212096, -33760778496, -30605104896, -27449431296, -21137997696, -17982324096, -14826650496, -12219271296, -2208988800 ], + "diffs" : [ -172704, -86304, 96, 86496, 172896, 259296, 345696, 432096, 518496, 604896, 691296, 777696, 864096, 96, 0 ] +}, { + "tz" : "Pacific/Gambier", + "switches" : [ -62135564412, -59006329212, -55850655612, -52694982012, -46383548412, -43227874812, -40072201212, -33760767612, -30605094012, -27449420412, -21137986812, -17982313212, -14826639612, -12219260412, -2208988800 ], + "diffs" : [ -172788, -86388, 12, 86412, 172812, 259212, 345612, 432012, 518412, 604812, 691212, 777612, 864012, 12, 0 ] +}, { + "tz" : "Pacific/Guadalcanal", + "switches" : [ -62135635188, -59006399988, -55850726388, -52695052788, -46383619188, -43227945588, -40072271988, -33760838388, -30605164788, -27449491188, -21138057588, -17982383988, -14826710388, -12219331188, -2208988800 ], + "diffs" : [ -174012, -87612, -1212, 85188, 171588, 257988, 344388, 430788, 517188, 603588, 689988, 776388, 862788, -1212, 0 ] +}, { + "tz" : "Pacific/Guam", + "switches" : [ -62135545140, -59006309940, -55850636340, -52694962740, -46383529140, -43227855540, -40072181940, -33760748340, -30605074740, -27449401140, -21137967540, -17982293940, -14826620340, -12219241140, -3944626740, -2208988800 ], + "diffs" : [ -260460, -174060, -87660, -1260, 85140, 171540, 257940, 344340, 430740, 517140, 603540, 689940, 776340, -87660, -1260, 0 ] +}, { + "tz" : "Pacific/Honolulu", + "switches" : [ -62135558914, -59006323714, -55850650114, -52694976514, -46383542914, -43227869314, -40072195714, -33760762114, -30605088514, -27449414914, -21137981314, -17982307714, -14826634114, -12219254914, -2334101314, -2208988800 ], + "diffs" : [ -174686, -88286, -1886, 84514, 170914, 257314, 343714, 430114, 516514, 602914, 689314, 775714, 862114, -1886, -1800, 0 ] +}, { + "tz" : "Pacific/Johnston", + "switches" : [ -62135558914, -59006323714, -55850650114, -52694976514, -46383542914, -43227869314, -40072195714, -33760762114, -30605088514, -27449414914, -21137981314, -17982307714, -14826634114, -12219254914, -2334101314, -2208988800 ], + "diffs" : [ -174686, -88286, -1886, 84514, 170914, 257314, 343714, 430114, 516514, 602914, 689314, 775714, 862114, -1886, -1800, 0 ] +}, { + "tz" : "Pacific/Kiritimati", + "switches" : [ -62135559040, -59006323840, -55850650240, -52694976640, -46383543040, -43227869440, -40072195840, -33760762240, -30605088640, -27449415040, -21137981440, -17982307840, -14826634240, -12219255040, -2208988800 ], + "diffs" : [ -260960, -174560, -88160, -1760, 84640, 171040, 257440, 343840, 430240, 516640, 603040, 689440, 775840, -88160, 0 ] +}, { + "tz" : "Pacific/Kosrae", + "switches" : [ -62135549516, -59006314316, -55850640716, -52694967116, -46383533516, -43227859916, -40072186316, -33760752716, -30605079116, -27449405516, -21137971916, -17982298316, -14826624716, -12219245516, -3944631116, -2208988800 ], + "diffs" : [ -259684, -173284, -86884, -484, 85916, 172316, 258716, 345116, 431516, 517916, 604316, 690716, 777116, -86884, -484, 0 ] +}, { + "tz" : "Pacific/Kwajalein", + "switches" : [ -62135636960, -59006401760, -55850728160, -52695054560, -46383620960, -43227947360, -40072273760, -33760840160, -30605166560, -27449492960, -21138059360, -17982385760, -14826712160, -12219332960, -2208988800 ], + "diffs" : [ -175840, -89440, -3040, 83360, 169760, 256160, 342560, 428960, 515360, 601760, 688160, 774560, 860960, -3040, 0 ] +}, { + "tz" : "Pacific/Majuro", + "switches" : [ -62135637888, -59006402688, -55850729088, -52695055488, -46383621888, -43227948288, -40072274688, -33760841088, -30605167488, -27449493888, -21138060288, -17982386688, -14826713088, -12219333888, -2208988800 ], + "diffs" : [ -174912, -88512, -2112, 84288, 170688, 257088, 343488, 429888, 516288, 602688, 689088, 775488, 861888, -2112, 0 ] +}, { + "tz" : "Pacific/Marquesas", + "switches" : [ -62135563320, -59006328120, -55850654520, -52694980920, -46383547320, -43227873720, -40072200120, -33760766520, -30605092920, -27449419320, -21137985720, -17982312120, -14826638520, -12219259320, -2208988800 ], + "diffs" : [ -172080, -85680, 720, 87120, 173520, 259920, 346320, 432720, 519120, 605520, 691920, 778320, 864720, 720, 0 ] +}, { + "tz" : "Pacific/Midway", + "switches" : [ -62135642232, -59006407032, -55850733432, -52695059832, -46383626232, -43227952632, -40072279032, -33760845432, -30605171832, -27449498232, -21138064632, -17982391032, -14826717432, -12219338232, -2445424632, -2208988800 ], + "diffs" : [ -87768, -1368, 85032, 171432, 257832, 344232, 430632, 517032, 603432, 689832, 776232, 862632, 949032, 85032, -1368, 0 ] +}, { + "tz" : "Pacific/Nauru", + "switches" : [ -62135636860, -59006401660, -55850728060, -52695054460, -46383620860, -43227947260, -40072273660, -33760840060, -30605166460, -27449492860, -21138059260, -17982385660, -14826712060, -12219332860, -2208988800 ], + "diffs" : [ -175940, -89540, -3140, 83260, 169660, 256060, 342460, 428860, 515260, 601660, 688060, 774460, 860860, -3140, 0 ] +}, { + "tz" : "Pacific/Niue", + "switches" : [ -62135556020, -59006320820, -55850647220, -52694973620, -46383540020, -43227866420, -40072192820, -33760759220, -30605085620, -27449412020, -21137978420, -17982304820, -14826631220, -12219252020, -2208988800 ], + "diffs" : [ -173980, -87580, -1180, 85220, 171620, 258020, 344420, 430820, 517220, 603620, 690020, 776420, 862820, -1180, 0 ] +}, { + "tz" : "Pacific/Norfolk", + "switches" : [ -62135637112, -59006401912, -55850728312, -52695054712, -46383621112, -43227947512, -40072273912, -33760840312, -30605166712, -27449493112, -21138059512, -17982385912, -14826712312, -12219333112, -2208988800 ], + "diffs" : [ -172088, -85688, 712, 87112, 173512, 259912, 346312, 432712, 519112, 605512, 691912, 778312, 864712, 712, 0 ] +}, { + "tz" : "Pacific/Noumea", + "switches" : [ -62135636748, -59006401548, -55850727948, -52695054348, -46383620748, -43227947148, -40072273548, -33760839948, -30605166348, -27449492748, -21138059148, -17982385548, -14826711948, -12219332748, -2208988800 ], + "diffs" : [ -172452, -86052, 348, 86748, 173148, 259548, 345948, 432348, 518748, 605148, 691548, 777948, 864348, 348, 0 ] +}, { + "tz" : "Pacific/Pago_Pago", + "switches" : [ -62135642232, -59006407032, -55850733432, -52695059832, -46383626232, -43227952632, -40072279032, -33760845432, -30605171832, -27449498232, -21138064632, -17982391032, -14826717432, -12219338232, -2445424632, -2208988800 ], + "diffs" : [ -87768, -1368, 85032, 171432, 257832, 344232, 430632, 517032, 603432, 689832, 776232, 862632, 949032, 85032, -1368, 0 ] +}, { + "tz" : "Pacific/Palau", + "switches" : [ -62135542676, -59006307476, -55850633876, -52694960276, -46383526676, -43227853076, -40072179476, -33760745876, -30605072276, -27449398676, -21137965076, -17982291476, -14826617876, -12219238676, -3944624276, -2208988800 ], + "diffs" : [ -259324, -172924, -86524, -124, 86276, 172676, 259076, 345476, 431876, 518276, 604676, 691076, 777476, -86524, -124, 0 ] +}, { + "tz" : "Pacific/Pitcairn", + "switches" : [ -62135565580, -59006330380, -55850656780, -52694983180, -46383549580, -43227875980, -40072202380, -33760768780, -30605095180, -27449421580, -21137987980, -17982314380, -14826640780, -12219261580, -2208988800 ], + "diffs" : [ -175220, -88820, -2420, 83980, 170380, 256780, 343180, 429580, 515980, 602380, 688780, 775180, 861580, -2420, 0 ] +}, { + "tz" : "Pacific/Pohnpei", + "switches" : [ -62135548372, -59006313172, -55850639572, -52694965972, -46383532372, -43227858772, -40072185172, -33760751572, -30605077972, -27449404372, -21137970772, -17982297172, -14826623572, -12219244372, -3944629972, -2208988800 ], + "diffs" : [ -260828, -174428, -88028, -1628, 84772, 171172, 257572, 343972, 430372, 516772, 603172, 689572, 775972, -88028, -1628, 0 ] +}, { + "tz" : "Pacific/Ponape", + "switches" : [ -62135548372, -59006313172, -55850639572, -52694965972, -46383532372, -43227858772, -40072185172, -33760751572, -30605077972, -27449404372, -21137970772, -17982297172, -14826623572, -12219244372, -3944629972, -2208988800 ], + "diffs" : [ -260828, -174428, -88028, -1628, 84772, 171172, 257572, 343972, 430372, 516772, 603172, 689572, 775972, -88028, -1628, 0 ] +}, { + "tz" : "Pacific/Port_Moresby", + "switches" : [ -62135632120, -59006396920, -55850723320, -52695049720, -46383616120, -43227942520, -40072268920, -33760835320, -30605161720, -27449488120, -21138054520, -17982380920, -14826707320, -12219328120, -2840176120, -2366790512 ], + "diffs" : [ -173480, -87080, -680, 85720, 172120, 258520, 344920, 431320, 517720, 604120, 690520, 776920, 863320, -680, -688, 0 ] +}, { + "tz" : "Pacific/Rarotonga", + "switches" : [ -62135558456, -59006323256, -55850649656, -52694976056, -46383542456, -43227868856, -40072195256, -33760761656, -30605088056, -27449414456, -21137980856, -17982307256, -14826633656, -12219254456, -2208988800 ], + "diffs" : [ -175144, -88744, -2344, 84056, 170456, 256856, 343256, 429656, 516056, 602456, 688856, 775256, 861656, -2344, 0 ] +}, { + "tz" : "Pacific/Saipan", + "switches" : [ -62135545140, -59006309940, -55850636340, -52694962740, -46383529140, -43227855540, -40072181940, -33760748340, -30605074740, -27449401140, -21137967540, -17982293940, -14826620340, -12219241140, -3944626740, -2208988800 ], + "diffs" : [ -260460, -174060, -87660, -1260, 85140, 171540, 257940, 344340, 430740, 517140, 603540, 689940, 776340, -87660, -1260, 0 ] +}, { + "tz" : "Pacific/Samoa", + "switches" : [ -62135642232, -59006407032, -55850733432, -52695059832, -46383626232, -43227952632, -40072279032, -33760845432, -30605171832, -27449498232, -21138064632, -17982391032, -14826717432, -12219338232, -2445424632, -2208988800 ], + "diffs" : [ -87768, -1368, 85032, 171432, 257832, 344232, 430632, 517032, 603432, 689832, 776232, 862632, 949032, 85032, -1368, 0 ] +}, { + "tz" : "Pacific/Tahiti", + "switches" : [ -62135560904, -59006325704, -55850652104, -52694978504, -46383544904, -43227871304, -40072197704, -33760764104, -30605090504, -27449416904, -21137983304, -17982309704, -14826636104, -12219256904, -2208988800 ], + "diffs" : [ -172696, -86296, 104, 86504, 172904, 259304, 345704, 432104, 518504, 604904, 691304, 777704, 864104, 104, 0 ] +}, { + "tz" : "Pacific/Tarawa", + "switches" : [ -62135638324, -59006403124, -55850729524, -52695055924, -46383622324, -43227948724, -40072275124, -33760841524, -30605167924, -27449494324, -21138060724, -17982387124, -14826713524, -12219334324, -2208988800 ], + "diffs" : [ -174476, -88076, -1676, 84724, 171124, 257524, 343924, 430324, 516724, 603124, 689524, 775924, 862324, -1676, 0 ] +}, { + "tz" : "Pacific/Tongatapu", + "switches" : [ -62135641160, -59006405960, -55850732360, -52695058760, -46383625160, -43227951560, -40072277960, -33760844360, -30605170760, -27449497160, -21138063560, -17982389960, -14826716360, -12219337160, -2208988800 ], + "diffs" : [ -175240, -88840, -2440, 83960, 170360, 256760, 343160, 429560, 515960, 602360, 688760, 775160, 861560, -2440, 0 ] +}, { + "tz" : "Pacific/Truk", + "switches" : [ -62135546828, -59006311628, -55850638028, -52694964428, -46383530828, -43227857228, -40072183628, -33760750028, -30605076428, -27449402828, -21137969228, -17982295628, -14826622028, -12219242828, -3944628428, -2208988800 ], + "diffs" : [ -258772, -172372, -85972, 428, 86828, 173228, 259628, 346028, 432428, 518828, 605228, 691628, 778028, -85972, 428, 0 ] +}, { + "tz" : "Pacific/Wake", + "switches" : [ -62135636788, -59006401588, -55850727988, -52695054388, -46383620788, -43227947188, -40072273588, -33760839988, -30605166388, -27449492788, -21138059188, -17982385588, -14826711988, -12219332788, -2208988800 ], + "diffs" : [ -176012, -89612, -3212, 83188, 169588, 255988, 342388, 428788, 515188, 601588, 687988, 774388, 860788, -3212, 0 ] +}, { + "tz" : "Pacific/Wallis", + "switches" : [ -62135640920, -59006405720, -55850732120, -52695058520, -46383624920, -43227951320, -40072277720, -33760844120, -30605170520, -27449496920, -21138063320, -17982389720, -14826716120, -12219336920, -2208988800 ], + "diffs" : [ -171880, -85480, 920, 87320, 173720, 260120, 346520, 432920, 519320, 605720, 692120, 778520, 864920, 920, 0 ] +}, { + "tz" : "Pacific/Yap", + "switches" : [ -62135546828, -59006311628, -55850638028, -52694964428, -46383530828, -43227857228, -40072183628, -33760750028, -30605076428, -27449402828, -21137969228, -17982295628, -14826622028, -12219242828, -3944628428, -2208988800 ], + "diffs" : [ -258772, -172372, -85972, 428, 86828, 173228, 259628, 346028, 432428, 518828, 605228, 691628, 778028, -85972, 428, 0 ] +}, { + "tz" : "Poland", + "switches" : [ -62135601840, -59006366640, -55850693040, -52695019440, -46383585840, -43227912240, -40072238640, -33760805040, -30605131440, -27449457840, -21138024240, -17982350640, -14826677040, -12219297840, -2208988800 ], + "diffs" : [ -171360, -84960, 1440, 87840, 174240, 260640, 347040, 433440, 519840, 606240, 692640, 779040, 865440, 1440, 0 ] +}, { + "tz" : "Portugal", + "switches" : [ -62135594595, -59006359395, -55850685795, -52695012195, -46383578595, -43227904995, -40072231395, -33760797795, -30605124195, -27449450595, -21138016995, -17982343395, -14826669795, -12219290595, -2208988800 ], + "diffs" : [ -175005, -88605, -2205, 84195, 170595, 256995, 343395, 429795, 516195, 602595, 688995, 775395, 861795, -2205, 0 ] +}, { + "tz" : "ROK", + "switches" : [ -62135627272, -59006392072, -55850718472, -52695044872, -46383611272, -43227937672, -40072264072, -33760830472, -30605156872, -27449483272, -21138049672, -17982376072, -14826702472, -12219323272, -2208988800 ], + "diffs" : [ -174728, -88328, -1928, 84472, 170872, 257272, 343672, 430072, 516472, 602872, 689272, 775672, 862072, -1928, 0 ] +}, { + "tz" : "Singapore", + "switches" : [ -62135621725, -59006386525, -55850712925, -52695039325, -46383605725, -43227932125, -40072258525, -33760824925, -30605151325, -27449477725, -21138044125, -17982370525, -14826696925, -12219317725, -2208988800 ], + "diffs" : [ -176675, -90275, -3875, 82525, 168925, 255325, 341725, 428125, 514525, 600925, 687325, 773725, 860125, -3875, 0 ] +}, { + "tz" : "SystemV/AST4", + "switches" : [ -62135582400, -59006347200, -55850673600, -52695000000, -46383566400, -43227892800, -40072219200, -33760785600, -30605112000, -27449438400, -21138004800, -17982331200, -14826657600, -12219278400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/AST4ADT", + "switches" : [ -62135582400, -59006347200, -55850673600, -52695000000, -46383566400, -43227892800, -40072219200, -33760785600, -30605112000, -27449438400, -21138004800, -17982331200, -14826657600, -12219278400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/CST6", + "switches" : [ -62135575200, -59006340000, -55850666400, -52694992800, -46383559200, -43227885600, -40072212000, -33760778400, -30605104800, -27449431200, -21137997600, -17982324000, -14826650400, -12219271200 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/CST6CDT", + "switches" : [ -62135575200, -59006340000, -55850666400, -52694992800, -46383559200, -43227885600, -40072212000, -33760778400, -30605104800, -27449431200, -21137997600, -17982324000, -14826650400, -12219271200 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/EST5", + "switches" : [ -62135578800, -59006343600, -55850670000, -52694996400, -46383562800, -43227889200, -40072215600, -33760782000, -30605108400, -27449434800, -21138001200, -17982327600, -14826654000, -12219274800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/EST5EDT", + "switches" : [ -62135578800, -59006343600, -55850670000, -52694996400, -46383562800, -43227889200, -40072215600, -33760782000, -30605108400, -27449434800, -21138001200, -17982327600, -14826654000, -12219274800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/HST10", + "switches" : [ -62135560800, -59006325600, -55850652000, -52694978400, -46383544800, -43227871200, -40072197600, -33760764000, -30605090400, -27449416800, -21137983200, -17982309600, -14826636000, -12219256800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/MST7", + "switches" : [ -62135571600, -59006336400, -55850662800, -52694989200, -46383555600, -43227882000, -40072208400, -33760774800, -30605101200, -27449427600, -21137994000, -17982320400, -14826646800, -12219267600 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/MST7MDT", + "switches" : [ -62135571600, -59006336400, -55850662800, -52694989200, -46383555600, -43227882000, -40072208400, -33760774800, -30605101200, -27449427600, -21137994000, -17982320400, -14826646800, -12219267600 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/PST8", + "switches" : [ -62135568000, -59006332800, -55850659200, -52694985600, -46383552000, -43227878400, -40072204800, -33760771200, -30605097600, -27449424000, -21137990400, -17982316800, -14826643200, -12219264000 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/PST8PDT", + "switches" : [ -62135568000, -59006332800, -55850659200, -52694985600, -46383552000, -43227878400, -40072204800, -33760771200, -30605097600, -27449424000, -21137990400, -17982316800, -14826643200, -12219264000 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/YST9", + "switches" : [ -62135564400, -59006329200, -55850655600, -52694982000, -46383548400, -43227874800, -40072201200, -33760767600, -30605094000, -27449420400, -21137986800, -17982313200, -14826639600, -12219260400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "SystemV/YST9YDT", + "switches" : [ -62135564400, -59006329200, -55850655600, -52694982000, -46383548400, -43227874800, -40072201200, -33760767600, -30605094000, -27449420400, -21137986800, -17982313200, -14826639600, -12219260400 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Turkey", + "switches" : [ -62135603752, -59006368552, -55850694952, -52695021352, -46383587752, -43227914152, -40072240552, -33760806952, -30605133352, -27449459752, -21138026152, -17982352552, -14826678952, -12219299752, -2840147752, -2208988800 ], + "diffs" : [ -176648, -90248, -3848, 82552, 168952, 255352, 341752, 428152, 514552, 600952, 687352, 773752, 860152, -3848, -3784, 0 ] +}, { + "tz" : "UCT", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "US/Alaska", + "switches" : [ -62135647224, -59006412024, -55850738424, -52695064824, -46383631224, -43227957624, -40072284024, -33760850424, -30605176824, -27449503224, -21138069624, -17982396024, -14826722424, -12219343224, -3225223727, -2208988800 ], + "diffs" : [ -89976, -3576, 82824, 169224, 255624, 342024, 428424, 514824, 601224, 687624, 774024, 860424, 946824, 82824, -3576, 0 ] +}, { + "tz" : "US/Aleutian", + "switches" : [ -62135640802, -59006405602, -55850732002, -52695058402, -46383624802, -43227951202, -40072277602, -33760844002, -30605170402, -27449496802, -21138063202, -17982389602, -14826716002, -12219336802, -3225223727, -2208988800 ], + "diffs" : [ -92798, -6398, 80002, 166402, 252802, 339202, 425602, 512002, 598402, 684802, 771202, 857602, 944002, 80002, -6398, 0 ] +}, { + "tz" : "US/Arizona", + "switches" : [ -62135569902, -59006334702, -55850661102, -52694987502, -46383553902, -43227880302, -40072206702, -33760773102, -30605099502, -27449425902, -21137992302, -17982318702, -14826645102, -12219265902, -2717643600 ], + "diffs" : [ -174498, -88098, -1698, 84702, 171102, 257502, 343902, 430302, 516702, 603102, 689502, 775902, 862302, -1698, 0 ] +}, { + "tz" : "US/Central", + "switches" : [ -62135575764, -59006340564, -55850666964, -52694993364, -46383559764, -43227886164, -40072212564, -33760778964, -30605105364, -27449431764, -21137998164, -17982324564, -14826650964, -12219271764, -2717647200 ], + "diffs" : [ -172236, -85836, 564, 86964, 173364, 259764, 346164, 432564, 518964, 605364, 691764, 778164, 864564, 564, 0 ] +}, { + "tz" : "US/East-Indiana", + "switches" : [ -62135576122, -59006340922, -55850667322, -52694993722, -46383560122, -43227886522, -40072212922, -33760779322, -30605105722, -27449432122, -21137998522, -17982324922, -14826651322, -12219272122, -2717647200, -2208988800 ], + "diffs" : [ -175478, -89078, -2678, 83722, 170122, 256522, 342922, 429322, 515722, 602122, 688522, 774922, 861322, -2678, -3600, 0 ] +}, { + "tz" : "US/Eastern", + "switches" : [ -62135579038, -59006343838, -55850670238, -52694996638, -46383563038, -43227889438, -40072215838, -33760782238, -30605108638, -27449435038, -21138001438, -17982327838, -14826654238, -12219275038, -2717650800 ], + "diffs" : [ -172562, -86162, 238, 86638, 173038, 259438, 345838, 432238, 518638, 605038, 691438, 777838, 864238, 238, 0 ] +}, { + "tz" : "US/Hawaii", + "switches" : [ -62135558914, -59006323714, -55850650114, -52694976514, -46383542914, -43227869314, -40072195714, -33760762114, -30605088514, -27449414914, -21137981314, -17982307714, -14826634114, -12219254914, -2334101314, -2208988800 ], + "diffs" : [ -174686, -88286, -1886, 84514, 170914, 257314, 343714, 430114, 516514, 602914, 689314, 775714, 862114, -1886, -1800, 0 ] +}, { + "tz" : "US/Indiana-Starke", + "switches" : [ -62135576010, -59006340810, -55850667210, -52694993610, -46383560010, -43227886410, -40072212810, -33760779210, -30605105610, -27449432010, -21137998410, -17982324810, -14826651210, -12219272010, -2717647200 ], + "diffs" : [ -171990, -85590, 810, 87210, 173610, 260010, 346410, 432810, 519210, 605610, 692010, 778410, 864810, 810, 0 ] +}, { + "tz" : "US/Michigan", + "switches" : [ -62135576869, -59006341669, -55850668069, -52694994469, -46383560869, -43227887269, -40072213669, -33760780069, -30605106469, -27449432869, -21137999269, -17982325669, -14826652069, -12219272869, -2208988800 ], + "diffs" : [ -174731, -88331, -1931, 84469, 170869, 257269, 343669, 430069, 516469, 602869, 689269, 775669, 862069, -1931, 0 ] +}, { + "tz" : "US/Mountain", + "switches" : [ -62135571604, -59006336404, -55850662804, -52694989204, -46383555604, -43227882004, -40072208404, -33760774804, -30605101204, -27449427604, -21137994004, -17982320404, -14826646804, -12219267604, -2717643600 ], + "diffs" : [ -172796, -86396, 4, 86404, 172804, 259204, 345604, 432004, 518404, 604804, 691204, 777604, 864004, 4, 0 ] +}, { + "tz" : "US/Pacific", + "switches" : [ -62135568422, -59006333222, -55850659622, -52694986022, -46383552422, -43227878822, -40072205222, -33760771622, -30605098022, -27449424422, -21137990822, -17982317222, -14826643622, -12219264422, -2717640000 ], + "diffs" : [ -172378, -85978, 422, 86822, 173222, 259622, 346022, 432422, 518822, 605222, 691622, 778022, 864422, 422, 0 ] +}, { + "tz" : "US/Pacific-New", + "switches" : [ -62135568422, -59006333222, -55850659622, -52694986022, -46383552422, -43227878822, -40072205222, -33760771622, -30605098022, -27449424422, -21137990822, -17982317222, -14826643622, -12219264422, -2717640000 ], + "diffs" : [ -172378, -85978, 422, 86822, 173222, 259622, 346022, 432422, 518822, 605222, 691622, 778022, 864422, 422, 0 ] +}, { + "tz" : "US/Samoa", + "switches" : [ -62135642232, -59006407032, -55850733432, -52695059832, -46383626232, -43227952632, -40072279032, -33760845432, -30605171832, -27449498232, -21138064632, -17982391032, -14826717432, -12219338232, -2445424632, -2208988800 ], + "diffs" : [ -87768, -1368, 85032, 171432, 257832, 344232, 430632, 517032, 603432, 689832, 776232, 862632, 949032, 85032, -1368, 0 ] +}, { + "tz" : "UTC", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Universal", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "W-SU", + "switches" : [ -62135605817, -59006370617, -55850697017, -52695023417, -46383589817, -43227916217, -40072242617, -33760809017, -30605135417, -27449461817, -21138028217, -17982354617, -14826681017, -12219301817, -2208988800 ], + "diffs" : [ -174583, -88183, -1783, 84617, 171017, 257417, 343817, 430217, 516617, 603017, 689417, 775817, 862217, -1783, 0 ] +}, { + "tz" : "WET", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +}, { + "tz" : "Zulu", + "switches" : [ -62135596800, -59006361600, -55850688000, -52695014400, -46383580800, -43227907200, -40072233600, -33760800000, -30605126400, -27449452800, -21138019200, -17982345600, -14826672000, -12219292800 ], + "diffs" : [ -172800, -86400, 0, 86400, 172800, 259200, 345600, 432000, 518400, 604800, 691200, 777600, 864000, 0 ] +} ] \ No newline at end of file diff --git a/sql/catalyst/src/main/resources/julian-gregorian-rebase-micros.json b/sql/catalyst/src/main/resources/julian-gregorian-rebase-micros.json new file mode 100644 index 0000000000000..b733ac43cbf1f --- /dev/null +++ b/sql/catalyst/src/main/resources/julian-gregorian-rebase-micros.json @@ -0,0 +1,2385 @@ +[ { + "tz" : "Africa/Abidjan", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Accra", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 172852, 86452, 52, -86348, -172748, -259148, -345548, -431948, -518348, -604748, -691148, -777548, -863948, 52, 0 ] +}, { + "tz" : "Africa/Addis_Ababa", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Africa/Algiers", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2486682111, -2208988800 ], + "diffs" : [ 175668, 89268, 2868, -83532, -169932, -256332, -342732, -429132, -515532, -601932, -688332, -774732, -861132, 2868, 3039, 0 ] +}, { + "tz" : "Africa/Asmara", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Africa/Asmera", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Africa/Bamako", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Bangui", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Banjul", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Bissau", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 176540, 90140, 3740, -82660, -169060, -255460, -341860, -428260, -514660, -601060, -687460, -773860, -860260, 3740, 0 ] +}, { + "tz" : "Africa/Blantyre", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172180, 85780, -620, -87020, -173420, -259820, -346220, -432620, -519020, -605420, -691820, -778220, -864620, -620, 0 ] +}, { + "tz" : "Africa/Brazzaville", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Bujumbura", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172180, 85780, -620, -87020, -173420, -259820, -346220, -432620, -519020, -605420, -691820, -778220, -864620, -620, 0 ] +}, { + "tz" : "Africa/Cairo", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172491, 86091, -309, -86709, -173109, -259509, -345909, -432309, -518709, -605109, -691509, -777909, -864309, -309, 0 ] +}, { + "tz" : "Africa/Ceuta", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 177676, 91276, 4876, -81524, -167924, -254324, -340724, -427124, -513524, -599924, -686324, -772724, -859124, 4876, 0 ] +}, { + "tz" : "Africa/Conakry", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Dakar", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Dar_es_Salaam", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Africa/Djibouti", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Africa/Douala", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Freetown", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Gaborone", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172180, 85780, -620, -87020, -173420, -259820, -346220, -432620, -519020, -605420, -691820, -778220, -864620, -620, 0 ] +}, { + "tz" : "Africa/Harare", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172180, 85780, -620, -87020, -173420, -259820, -346220, -432620, -519020, -605420, -691820, -778220, -864620, -620, 0 ] +}, { + "tz" : "Africa/Johannesburg", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2458174920, -2208988800 ], + "diffs" : [ 173280, 86880, 480, -85920, -172320, -258720, -345120, -431520, -517920, -604320, -690720, -777120, -863520, 480, 1800, 0 ] +}, { + "tz" : "Africa/Juba", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 176012, 89612, 3212, -83188, -169588, -255988, -342388, -428788, -515188, -601588, -687988, -774388, -860788, 3212, 0 ] +}, { + "tz" : "Africa/Kampala", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Africa/Khartoum", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172192, 85792, -608, -87008, -173408, -259808, -346208, -432608, -519008, -605408, -691808, -778208, -864608, -608, 0 ] +}, { + "tz" : "Africa/Kigali", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172180, 85780, -620, -87020, -173420, -259820, -346220, -432620, -519020, -605420, -691820, -778220, -864620, -620, 0 ] +}, { + "tz" : "Africa/Kinshasa", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Lagos", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Libreville", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Lome", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Luanda", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Lubumbashi", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172180, 85780, -620, -87020, -173420, -259820, -346220, -432620, -519020, -605420, -691820, -778220, -864620, -620, 0 ] +}, { + "tz" : "Africa/Lusaka", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172180, 85780, -620, -87020, -173420, -259820, -346220, -432620, -519020, -605420, -691820, -778220, -864620, -620, 0 ] +}, { + "tz" : "Africa/Malabo", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Maputo", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172180, 85780, -620, -87020, -173420, -259820, -346220, -432620, -519020, -605420, -691820, -778220, -864620, -620, 0 ] +}, { + "tz" : "Africa/Maseru", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2458174920, -2208988800 ], + "diffs" : [ 173280, 86880, 480, -85920, -172320, -258720, -345120, -431520, -517920, -604320, -690720, -777120, -863520, 480, 1800, 0 ] +}, { + "tz" : "Africa/Mbabane", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2458174920, -2208988800 ], + "diffs" : [ 173280, 86880, 480, -85920, -172320, -258720, -345120, -431520, -517920, -604320, -690720, -777120, -863520, 480, 1800, 0 ] +}, { + "tz" : "Africa/Mogadishu", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Africa/Monrovia", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 175388, 88988, 2588, -83812, -170212, -256612, -343012, -429412, -515812, -602212, -688612, -775012, -861412, 2588, 0 ] +}, { + "tz" : "Africa/Nairobi", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Africa/Ndjamena", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 172788, 86388, -12, -86412, -172812, -259212, -345612, -432012, -518412, -604812, -691212, -777612, -864012, -12, 0 ] +}, { + "tz" : "Africa/Niamey", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Nouakchott", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Ouagadougou", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Porto-Novo", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Africa/Sao_Tome", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2713914221, -2208988800 ], + "diffs" : [ 171184, 84784, -1616, -88016, -174416, -260816, -347216, -433616, -520016, -606416, -692816, -779216, -865616, -1616, 2205, 0 ] +}, { + "tz" : "Africa/Timbuktu", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Africa/Tripoli", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 176836, 90436, 4036, -82364, -168764, -255164, -341564, -427964, -514364, -600764, -687164, -773564, -859964, 4036, 0 ] +}, { + "tz" : "Africa/Tunis", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2797205483, -2208988800 ], + "diffs" : [ 173956, 87556, 1156, -85244, -171644, -258044, -344444, -430844, -517244, -603644, -690044, -776444, -862844, 1156, 3039, 0 ] +}, { + "tz" : "Africa/Windhoek", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2458172304, -2208988800 ], + "diffs" : [ 175896, 89496, 3096, -83304, -169704, -256104, -342504, -428904, -515304, -601704, -688104, -774504, -860904, 3096, 1800, 0 ] +}, { + "tz" : "America/Adak", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800, -3225230125, -2208988800 ], + "diffs" : [ 92798, 6398, -80002, -166402, -252802, -339202, -425602, -512002, -598402, -684802, -771202, -857602, -944002, -80002, 6398, 0 ] +}, { + "tz" : "America/Anchorage", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400, -3225227303, -2208988800 ], + "diffs" : [ 89976, 3576, -82824, -169224, -255624, -342024, -428424, -514824, -601224, -687624, -774024, -860424, -946824, -82824, 3576, 0 ] +}, { + "tz" : "America/Anguilla", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Antigua", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Araguaina", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 173568, 87168, 768, -85632, -172032, -258432, -344832, -431232, -517632, -604032, -690432, -776832, -863232, 768, 0 ] +}, { + "tz" : "America/Argentina/Buenos_Aires", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372102580, -2208988800 ], + "diffs" : [ 176028, 89628, 3228, -83172, -169572, -255972, -342372, -428772, -515172, -601572, -687972, -774372, -860772, 3228, 4608, 0 ] +}, { + "tz" : "America/Argentina/Catamarca", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100820, -2208988800 ], + "diffs" : [ 177788, 91388, 4988, -81412, -167812, -254212, -340612, -427012, -513412, -599812, -686212, -772612, -859012, 4988, 4608, 0 ] +}, { + "tz" : "America/Argentina/ComodRivadavia", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100820, -2208988800 ], + "diffs" : [ 177788, 91388, 4988, -81412, -167812, -254212, -340612, -427012, -513412, -599812, -686212, -772612, -859012, 4988, 4608, 0 ] +}, { + "tz" : "America/Argentina/Cordoba", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 177408, 91008, 4608, -81792, -168192, -254592, -340992, -427392, -513792, -600192, -686592, -772992, -859392, 4608, 0 ] +}, { + "tz" : "America/Argentina/Jujuy", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100936, -2208988800 ], + "diffs" : [ 177672, 91272, 4872, -81528, -167928, -254328, -340728, -427128, -513528, -599928, -686328, -772728, -859128, 4872, 4608, 0 ] +}, { + "tz" : "America/Argentina/La_Rioja", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100564, -2208988800 ], + "diffs" : [ 178044, 91644, 5244, -81156, -167556, -253956, -340356, -426756, -513156, -599556, -685956, -772356, -858756, 5244, 4608, 0 ] +}, { + "tz" : "America/Argentina/Mendoza", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100092, -2208988800 ], + "diffs" : [ 178516, 92116, 5716, -80684, -167084, -253484, -339884, -426284, -512684, -599084, -685484, -771884, -858284, 5716, 4608, 0 ] +}, { + "tz" : "America/Argentina/Rio_Gallegos", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372099996, -2208988800 ], + "diffs" : [ 178612, 92212, 5812, -80588, -166988, -253388, -339788, -426188, -512588, -598988, -685388, -771788, -858188, 5812, 4608, 0 ] +}, { + "tz" : "America/Argentina/Salta", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100908, -2208988800 ], + "diffs" : [ 177700, 91300, 4900, -81500, -167900, -254300, -340700, -427100, -513500, -599900, -686300, -772700, -859100, 4900, 4608, 0 ] +}, { + "tz" : "America/Argentina/San_Juan", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100164, -2208988800 ], + "diffs" : [ 178444, 92044, 5644, -80756, -167156, -253556, -339956, -426356, -512756, -599156, -685556, -771956, -858356, 5644, 4608, 0 ] +}, { + "tz" : "America/Argentina/San_Luis", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100684, -2208988800 ], + "diffs" : [ 177924, 91524, 5124, -81276, -167676, -254076, -340476, -426876, -513276, -599676, -686076, -772476, -858876, 5124, 4608, 0 ] +}, { + "tz" : "America/Argentina/Tucuman", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100956, -2208988800 ], + "diffs" : [ 177652, 91252, 4852, -81548, -167948, -254348, -340748, -427148, -513548, -599948, -686348, -772748, -859148, 4852, 4608, 0 ] +}, { + "tz" : "America/Argentina/Ushuaia", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100216, -2208988800 ], + "diffs" : [ 178392, 91992, 5592, -80808, -167208, -253608, -340008, -426408, -512808, -599208, -685608, -772008, -858408, 5592, 4608, 0 ] +}, { + "tz" : "America/Aruba", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 174947, 88547, 2147, -84253, -170653, -257053, -343453, -429853, -516253, -602653, -689053, -775453, -861853, 2147, 0 ] +}, { + "tz" : "America/Asuncion", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 172240, 85840, -560, -86960, -173360, -259760, -346160, -432560, -518960, -605360, -691760, -778160, -864560, -560, 0 ] +}, { + "tz" : "America/Atikokan", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2366736812, -2208988800 ], + "diffs" : [ 176788, 90388, 3988, -82412, -168812, -255212, -341612, -428012, -514412, -600812, -687212, -773612, -860012, 3988, 3600, 0 ] +}, { + "tz" : "America/Atka", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800, -3225230125, -2208988800 ], + "diffs" : [ 92798, 6398, -80002, -166402, -252802, -339202, -425602, -512002, -598402, -684802, -771202, -857602, -944002, -80002, 6398, 0 ] +}, { + "tz" : "America/Bahia", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 171244, 84844, -1556, -87956, -174356, -260756, -347156, -433556, -519956, -606356, -692756, -779156, -865556, -1556, 0 ] +}, { + "tz" : "America/Bahia_Banderas", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 176460, 90060, 3660, -82740, -169140, -255540, -341940, -428340, -514740, -601140, -687540, -773940, -860340, 3660, 0 ] +}, { + "tz" : "America/Barbados", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 172709, 86309, -91, -86491, -172891, -259291, -345691, -432091, -518491, -604891, -691291, -777691, -864091, -91, 0 ] +}, { + "tz" : "America/Belem", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 173636, 87236, 836, -85564, -171964, -258364, -344764, -431164, -517564, -603964, -690364, -776764, -863164, 836, 0 ] +}, { + "tz" : "America/Belize", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 172368, 85968, -432, -86832, -173232, -259632, -346032, -432432, -518832, -605232, -691632, -778032, -864432, -432, 0 ] +}, { + "tz" : "America/Blanc-Sablon", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2713896692 ], + "diffs" : [ 172108, 85708, -692, -87092, -173492, -259892, -346292, -432692, -519092, -605492, -691892, -778292, -864692, -692, 0 ] +}, { + "tz" : "America/Boa_Vista", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 172960, 86560, 160, -86240, -172640, -259040, -345440, -431840, -518240, -604640, -691040, -777440, -863840, 160, 0 ] +}, { + "tz" : "America/Bogota", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 172576, 86176, -224, -86624, -173024, -259424, -345824, -432224, -518624, -605024, -691424, -777824, -864224, -224, 0 ] +}, { + "tz" : "America/Boise", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2717643600, -2208988800 ], + "diffs" : [ 175489, 89089, 2689, -83711, -170111, -256511, -342911, -429311, -515711, -602111, -688511, -774911, -861311, 2689, 3600, 0 ] +}, { + "tz" : "America/Buenos_Aires", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372102580, -2208988800 ], + "diffs" : [ 176028, 89628, 3228, -83172, -169572, -255972, -342372, -428772, -515172, -601572, -687972, -774372, -860772, 3228, 4608, 0 ] +}, { + "tz" : "America/Cambridge_Bay", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 147600, 61200, -25200, -111600, -198000, -284400, -370800, -457200, -543600, -630000, -716400, -802800, -889200, -25200, 0 ] +}, { + "tz" : "America/Campo_Grande", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 171508, 85108, -1292, -87692, -174092, -260492, -346892, -433292, -519692, -606092, -692492, -778892, -865292, -1292, 0 ] +}, { + "tz" : "America/Cancun", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 175624, 89224, 2824, -83576, -169976, -256376, -342776, -429176, -515576, -601976, -688376, -774776, -861176, 2824, 0 ] +}, { + "tz" : "America/Caracas", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2524507196, -2208988800 ], + "diffs" : [ 174464, 88064, 1664, -84736, -171136, -257536, -343936, -430336, -516736, -603136, -689536, -775936, -862336, 1664, 1660, 0 ] +}, { + "tz" : "America/Catamarca", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100820, -2208988800 ], + "diffs" : [ 177788, 91388, 4988, -81412, -167812, -254212, -340612, -427012, -513412, -599812, -686212, -772612, -859012, 4988, 4608, 0 ] +}, { + "tz" : "America/Cayenne", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 174560, 88160, 1760, -84640, -171040, -257440, -343840, -430240, -516640, -603040, -689440, -775840, -862240, 1760, 0 ] +}, { + "tz" : "America/Cayman", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2524503688, -2208988800 ], + "diffs" : [ 173888, 87488, 1088, -85312, -171712, -258112, -344512, -430912, -517312, -603712, -690112, -776512, -862912, 1088, 1176, 0 ] +}, { + "tz" : "America/Chicago", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2717647200 ], + "diffs" : [ 172236, 85836, -564, -86964, -173364, -259764, -346164, -432564, -518964, -605364, -691764, -778164, -864564, -564, 0 ] +}, { + "tz" : "America/Chihuahua", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 173060, 86660, 260, -86140, -172540, -258940, -345340, -431740, -518140, -604540, -690940, -777340, -863740, 260, 0 ] +}, { + "tz" : "America/Coral_Harbour", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2366736812, -2208988800 ], + "diffs" : [ 176788, 90388, 3988, -82412, -168812, -255212, -341612, -428012, -514412, -600812, -687212, -773612, -860012, 3988, 3600, 0 ] +}, { + "tz" : "America/Cordoba", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 177408, 91008, 4608, -81792, -168192, -254592, -340992, -427392, -513792, -600192, -686592, -772992, -859392, 4608, 0 ] +}, { + "tz" : "America/Costa_Rica", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 171373, 84973, -1427, -87827, -174227, -260627, -347027, -433427, -519827, -606227, -692627, -779027, -865427, -1427, 0 ] +}, { + "tz" : "America/Creston", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2713882436 ], + "diffs" : [ 175564, 89164, 2764, -83636, -170036, -256436, -342836, -429236, -515636, -602036, -688436, -774836, -861236, 2764, 0 ] +}, { + "tz" : "America/Cuiaba", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 171860, 85460, -940, -87340, -173740, -260140, -346540, -432940, -519340, -605740, -692140, -778540, -864940, -940, 0 ] +}, { + "tz" : "America/Curacao", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 174947, 88547, 2147, -84253, -170653, -257053, -343453, -429853, -516253, -602653, -689053, -775453, -861853, 2147, 0 ] +}, { + "tz" : "America/Danmarkshavn", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 177280, 90880, 4480, -81920, -168320, -254720, -341120, -427520, -513920, -600320, -686720, -773120, -859520, 4480, 0 ] +}, { + "tz" : "America/Dawson", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2208988800 ], + "diffs" : [ 177460, 91060, 4660, -81740, -168140, -254540, -340940, -427340, -513740, -600140, -686540, -772940, -859340, 4660, 0 ] +}, { + "tz" : "America/Dawson_Creek", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2713885144, -2208988800 ], + "diffs" : [ 176456, 90056, 3656, -82744, -169144, -255544, -341944, -428344, -514744, -601144, -687544, -773944, -860344, 3656, 3600, 0 ] +}, { + "tz" : "America/Denver", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2717643600 ], + "diffs" : [ 172796, 86396, -4, -86404, -172804, -259204, -345604, -432004, -518404, -604804, -691204, -777604, -864004, -4, 0 ] +}, { + "tz" : "America/Detroit", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 174731, 88331, 1931, -84469, -170869, -257269, -343669, -430069, -516469, -602869, -689269, -775669, -862069, 1931, 0 ] +}, { + "tz" : "America/Dominica", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Edmonton", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 174832, 88432, 2032, -84368, -170768, -257168, -343568, -429968, -516368, -602768, -689168, -775568, -861968, 2032, 0 ] +}, { + "tz" : "America/Eirunepe", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 171568, 85168, -1232, -87632, -174032, -260432, -346832, -433232, -519632, -606032, -692432, -778832, -865232, -1232, 0 ] +}, { + "tz" : "America/El_Salvador", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 172608, 86208, -192, -86592, -172992, -259392, -345792, -432192, -518592, -604992, -691392, -777792, -864192, -192, 0 ] +}, { + "tz" : "America/Ensenada", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2208988800 ], + "diffs" : [ 172084, 85684, -716, -87116, -173516, -259916, -346316, -432716, -519116, -605516, -691916, -778316, -864716, -716, 0 ] +}, { + "tz" : "America/Fort_Nelson", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2713884553, -2208988800 ], + "diffs" : [ 177047, 90647, 4247, -82153, -168553, -254953, -341353, -427753, -514153, -600553, -686953, -773353, -859753, 4247, 3600, 0 ] +}, { + "tz" : "America/Fort_Wayne", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175478, 89078, 2678, -83722, -170122, -256522, -342922, -429322, -515722, -602122, -688522, -774922, -861322, 2678, 3600, 0 ] +}, { + "tz" : "America/Fortaleza", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 171240, 84840, -1560, -87960, -174360, -260760, -347160, -433560, -519960, -606360, -692760, -779160, -865560, -1560, 0 ] +}, { + "tz" : "America/Glace_Bay", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 172788, 86388, -12, -86412, -172812, -259212, -345612, -432012, -518412, -604812, -691212, -777612, -864012, -12, 0 ] +}, { + "tz" : "America/Godthab", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 174416, 88016, 1616, -84784, -171184, -257584, -343984, -430384, -516784, -603184, -689584, -775984, -862384, 1616, 0 ] +}, { + "tz" : "America/Goose_Bay", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2713894152, -2208988800 ], + "diffs" : [ 172900, 86500, 100, -86300, -172700, -259100, -345500, -431900, -518300, -604700, -691100, -777500, -863900, 100, -1748, 0 ] +}, { + "tz" : "America/Grand_Turk", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2524504958, -2208988800 ], + "diffs" : [ 171872, 85472, -928, -87328, -173728, -260128, -346528, -432928, -519328, -605728, -692128, -778528, -864928, -928, 430, 0 ] +}, { + "tz" : "America/Grenada", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Guadeloupe", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Guatemala", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 172924, 86524, 124, -86276, -172676, -259076, -345476, -431876, -518276, -604676, -691076, -777476, -863876, 124, 0 ] +}, { + "tz" : "America/Guayaquil", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2524503280, -2208988800 ], + "diffs" : [ 173960, 87560, 1160, -85240, -171640, -258040, -344440, -430840, -517240, -603640, -690040, -776440, -862840, 1160, 840, 0 ] +}, { + "tz" : "America/Guyana", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 172360, 85960, -440, -86840, -173240, -259640, -346040, -432440, -518840, -605240, -691640, -778040, -864440, -440, 0 ] +}, { + "tz" : "America/Halifax", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173664, 87264, 864, -85536, -171936, -258336, -344736, -431136, -517536, -603936, -690336, -776736, -863136, 864, 0 ] +}, { + "tz" : "America/Havana", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2524503608, -2208988800 ], + "diffs" : [ 174568, 88168, 1768, -84632, -171032, -257432, -343832, -430232, -516632, -603032, -689432, -775832, -862232, 1768, 1776, 0 ] +}, { + "tz" : "America/Hermosillo", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 174232, 87832, 1432, -84968, -171368, -257768, -344168, -430568, -516968, -603368, -689768, -776168, -862568, 1432, 0 ] +}, { + "tz" : "America/Indiana/Indianapolis", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175478, 89078, 2678, -83722, -170122, -256522, -342922, -429322, -515722, -602122, -688522, -774922, -861322, 2678, 3600, 0 ] +}, { + "tz" : "America/Indiana/Knox", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2717647200 ], + "diffs" : [ 171990, 85590, -810, -87210, -173610, -260010, -346410, -432810, -519210, -605610, -692010, -778410, -864810, -810, 0 ] +}, { + "tz" : "America/Indiana/Marengo", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175523, 89123, 2723, -83677, -170077, -256477, -342877, -429277, -515677, -602077, -688477, -774877, -861277, 2723, 3600, 0 ] +}, { + "tz" : "America/Indiana/Petersburg", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175747, 89347, 2947, -83453, -169853, -256253, -342653, -429053, -515453, -601853, -688253, -774653, -861053, 2947, 3600, 0 ] +}, { + "tz" : "America/Indiana/Tell_City", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2717647200 ], + "diffs" : [ 172023, 85623, -777, -87177, -173577, -259977, -346377, -432777, -519177, -605577, -691977, -778377, -864777, -777, 0 ] +}, { + "tz" : "America/Indiana/Vevay", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175216, 88816, 2416, -83984, -170384, -256784, -343184, -429584, -515984, -602384, -688784, -775184, -861584, 2416, 3600, 0 ] +}, { + "tz" : "America/Indiana/Vincennes", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175807, 89407, 3007, -83393, -169793, -256193, -342593, -428993, -515393, -601793, -688193, -774593, -860993, 3007, 3600, 0 ] +}, { + "tz" : "America/Indiana/Winamac", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175585, 89185, 2785, -83615, -170015, -256415, -342815, -429215, -515615, -602015, -688415, -774815, -861215, 2785, 3600, 0 ] +}, { + "tz" : "America/Indianapolis", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175478, 89078, 2678, -83722, -170122, -256522, -342922, -429322, -515722, -602122, -688522, -774922, -861322, 2678, 3600, 0 ] +}, { + "tz" : "America/Inuvik", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 147600, 61200, -25200, -111600, -198000, -284400, -370800, -457200, -543600, -630000, -716400, -802800, -889200, -25200, 0 ] +}, { + "tz" : "America/Iqaluit", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 154800, 68400, -18000, -104400, -190800, -277200, -363600, -450000, -536400, -622800, -709200, -795600, -882000, -18000, 0 ] +}, { + "tz" : "America/Jamaica", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 173230, 86830, 430, -85970, -172370, -258770, -345170, -431570, -517970, -604370, -690770, -777170, -863570, 430, 0 ] +}, { + "tz" : "America/Jujuy", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100936, -2208988800 ], + "diffs" : [ 177672, 91272, 4872, -81528, -167928, -254328, -340728, -427128, -513528, -599928, -686328, -772728, -859128, 4872, 4608, 0 ] +}, { + "tz" : "America/Juneau", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400, -3225223588, -2208988800 ], + "diffs" : [ 86261, -139, -86539, -172939, -259339, -345739, -432139, -518539, -604939, -691339, -777739, -864139, -950539, -86539, -139, 0 ] +}, { + "tz" : "America/Kentucky/Louisville", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175382, 88982, 2582, -83818, -170218, -256618, -343018, -429418, -515818, -602218, -688618, -775018, -861418, 2582, 3600, 0 ] +}, { + "tz" : "America/Kentucky/Monticello", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175164, 88764, 2364, -84036, -170436, -256836, -343236, -429636, -516036, -602436, -688836, -775236, -861636, 2364, 3600, 0 ] +}, { + "tz" : "America/Knox_IN", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2717647200 ], + "diffs" : [ 171990, 85590, -810, -87210, -173610, -260010, -346410, -432810, -519210, -605610, -692010, -778410, -864810, -810, 0 ] +}, { + "tz" : "America/Kralendijk", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 174947, 88547, 2147, -84253, -170653, -257053, -343453, -429853, -516253, -602653, -689053, -775453, -861853, 2147, 0 ] +}, { + "tz" : "America/La_Paz", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 174756, 88356, 1956, -84444, -170844, -257244, -343644, -430044, -516444, -602844, -689244, -775644, -862044, 1956, 0 ] +}, { + "tz" : "America/Lima", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2524503624, -2208988800 ], + "diffs" : [ 173292, 86892, 492, -85908, -172308, -258708, -345108, -431508, -517908, -604308, -690708, -777108, -863508, 492, 516, 0 ] +}, { + "tz" : "America/Los_Angeles", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2717640000 ], + "diffs" : [ 172378, 85978, -422, -86822, -173222, -259622, -346022, -432422, -518822, -605222, -691622, -778022, -864422, -422, 0 ] +}, { + "tz" : "America/Louisville", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175382, 88982, 2582, -83818, -170218, -256618, -343018, -429418, -515818, -602218, -688618, -775018, -861418, 2582, 3600, 0 ] +}, { + "tz" : "America/Lower_Princes", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 174947, 88547, 2147, -84253, -170653, -257053, -343453, -429853, -516253, -602653, -689053, -775453, -861853, 2147, 0 ] +}, { + "tz" : "America/Maceio", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 170572, 84172, -2228, -88628, -175028, -261428, -347828, -434228, -520628, -607028, -693428, -779828, -866228, -2228, 0 ] +}, { + "tz" : "America/Managua", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2524500004, -2208988800 ], + "diffs" : [ 171908, 85508, -892, -87292, -173692, -260092, -346492, -432892, -519292, -605692, -692092, -778492, -864892, -892, -888, 0 ] +}, { + "tz" : "America/Manaus", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 172804, 86404, 4, -86396, -172796, -259196, -345596, -431996, -518396, -604796, -691196, -777596, -863996, 4, 0 ] +}, { + "tz" : "America/Marigot", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Martinique", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173060, 86660, 260, -86140, -172540, -258940, -345340, -431740, -518140, -604540, -690940, -777340, -863740, 260, 0 ] +}, { + "tz" : "America/Matamoros", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 175200, 88800, 2400, -84000, -170400, -256800, -343200, -429600, -516000, -602400, -688800, -775200, -861600, 2400, 0 ] +}, { + "tz" : "America/Mazatlan", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 173140, 86740, 340, -86060, -172460, -258860, -345260, -431660, -518060, -604460, -690860, -777260, -863660, 340, 0 ] +}, { + "tz" : "America/Mendoza", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2372100092, -2208988800 ], + "diffs" : [ 178516, 92116, 5716, -80684, -167084, -253484, -339884, -426284, -512684, -599084, -685484, -771884, -858284, 5716, 4608, 0 ] +}, { + "tz" : "America/Menominee", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2659759773 ], + "diffs" : [ 172227, 85827, -573, -86973, -173373, -259773, -346173, -432573, -518973, -605373, -691773, -778173, -864573, -573, 0 ] +}, { + "tz" : "America/Merida", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 172708, 86308, -92, -86492, -172892, -259292, -345692, -432092, -518492, -604892, -691292, -777692, -864092, -92, 0 ] +}, { + "tz" : "America/Metlakatla", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400, -3225222905, -2208988800 ], + "diffs" : [ 85578, -822, -87222, -173622, -260022, -346422, -432822, -519222, -605622, -692022, -778422, -864822, -951222, -87222, -822, 0 ] +}, { + "tz" : "America/Mexico_City", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 174996, 88596, 2196, -84204, -170604, -257004, -343404, -429804, -516204, -602604, -689004, -775404, -861804, 2196, 0 ] +}, { + "tz" : "America/Miquelon", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 175480, 89080, 2680, -83720, -170120, -256520, -342920, -429320, -515720, -602120, -688520, -774920, -861320, 2680, 0 ] +}, { + "tz" : "America/Moncton", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2715885652, -2208988800 ], + "diffs" : [ 173948, 87548, 1148, -85252, -171652, -258052, -344452, -430852, -517252, -603652, -690052, -776452, -862852, 1148, 3600, 0 ] +}, { + "tz" : "America/Monterrey", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 175276, 88876, 2476, -83924, -170324, -256724, -343124, -429524, -515924, -602324, -688724, -775124, -861524, 2476, 0 ] +}, { + "tz" : "America/Montevideo", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 175491, 89091, 2691, -83709, -170109, -256509, -342909, -429309, -515709, -602109, -688509, -774909, -861309, 2691, 0 ] +}, { + "tz" : "America/Montreal", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2366736148 ], + "diffs" : [ 173852, 87452, 1052, -85348, -171748, -258148, -344548, -430948, -517348, -603748, -690148, -776548, -862948, 1052, 0 ] +}, { + "tz" : "America/Montserrat", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Nassau", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 173370, 86970, 570, -85830, -172230, -258630, -345030, -431430, -517830, -604230, -690630, -777030, -863430, 570, 0 ] +}, { + "tz" : "America/New_York", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800 ], + "diffs" : [ 172562, 86162, -238, -86638, -173038, -259438, -345838, -432238, -518638, -605038, -691438, -777838, -864238, -238, 0 ] +}, { + "tz" : "America/Nipigon", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2366734016 ], + "diffs" : [ 175984, 89584, 3184, -83216, -169616, -256016, -342416, -428816, -515216, -601616, -688016, -774416, -860816, 3184, 0 ] +}, { + "tz" : "America/Nome", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400, -3225231025, -2208988800 ], + "diffs" : [ 93698, 7298, -79102, -165502, -251902, -338302, -424702, -511102, -597502, -683902, -770302, -856702, -943102, -79102, 7298, 0 ] +}, { + "tz" : "America/Noronha", + "switches" : [ -62135762400, -59006440800, -55850680800, -52694920800, -46383400800, -43227640800, -40071880800, -33760360800, -30604600800, -27448840800, -21137320800, -17981560800, -14825800800, -12219285600, -2208988800 ], + "diffs" : [ 173380, 86980, 580, -85820, -172220, -258620, -345020, -431420, -517820, -604220, -690620, -777020, -863420, 580, 0 ] +}, { + "tz" : "America/North_Dakota/Beulah", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2717647200, -2208988800 ], + "diffs" : [ 175627, 89227, 2827, -83573, -169973, -256373, -342773, -429173, -515573, -601973, -688373, -774773, -861173, 2827, 3600, 0 ] +}, { + "tz" : "America/North_Dakota/Center", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2717647200, -2208988800 ], + "diffs" : [ 175512, 89112, 2712, -83688, -170088, -256488, -342888, -429288, -515688, -602088, -688488, -774888, -861288, 2712, 3600, 0 ] +}, { + "tz" : "America/North_Dakota/New_Salem", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2717647200, -2208988800 ], + "diffs" : [ 175539, 89139, 2739, -83661, -170061, -256461, -342861, -429261, -515661, -602061, -688461, -774861, -861261, 2739, 3600, 0 ] +}, { + "tz" : "America/Ojinaga", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 172660, 86260, -140, -86540, -172940, -259340, -345740, -432140, -518540, -604940, -691340, -777740, -864140, -140, 0 ] +}, { + "tz" : "America/Panama", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2524503688, -2208988800 ], + "diffs" : [ 173888, 87488, 1088, -85312, -171712, -258112, -344512, -430912, -517312, -603712, -690112, -776512, -862912, 1088, 1176, 0 ] +}, { + "tz" : "America/Pangnirtung", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 154800, 68400, -18000, -104400, -190800, -277200, -363600, -450000, -536400, -622800, -709200, -795600, -882000, -18000, 0 ] +}, { + "tz" : "America/Paramaribo", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 175240, 88840, 2440, -83960, -170360, -256760, -343160, -429560, -515960, -602360, -688760, -775160, -861560, 2440, 0 ] +}, { + "tz" : "America/Phoenix", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2717643600 ], + "diffs" : [ 174498, 88098, 1698, -84702, -171102, -257502, -343902, -430302, -516702, -603102, -689502, -775902, -862302, 1698, 0 ] +}, { + "tz" : "America/Port-au-Prince", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2524503580, -2208988800 ], + "diffs" : [ 172160, 85760, -640, -87040, -173440, -259840, -346240, -432640, -519040, -605440, -691840, -778240, -864640, -640, -660, 0 ] +}, { + "tz" : "America/Port_of_Spain", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Porto_Acre", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 171072, 84672, -1728, -88128, -174528, -260928, -347328, -433728, -520128, -606528, -692928, -779328, -865728, -1728, 0 ] +}, { + "tz" : "America/Porto_Velho", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173736, 87336, 936, -85464, -171864, -258264, -344664, -431064, -517464, -603864, -690264, -776664, -863064, 936, 0 ] +}, { + "tz" : "America/Puerto_Rico", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2233035335 ], + "diffs" : [ 174265, 87865, 1465, -84935, -171335, -257735, -344135, -430535, -516935, -603335, -689735, -776135, -862535, 1465, 0 ] +}, { + "tz" : "America/Punta_Arenas", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2524510746, -2208988800 ], + "diffs" : [ 179020, 92620, 6220, -80180, -166580, -252980, -339380, -425780, -512180, -598580, -684980, -771380, -857780, 6220, 6166, 0 ] +}, { + "tz" : "America/Rainy_River", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2366732504 ], + "diffs" : [ 173896, 87496, 1096, -85304, -171704, -258104, -344504, -430904, -517304, -603704, -690104, -776504, -862904, 1096, 0 ] +}, { + "tz" : "America/Rankin_Inlet", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 151200, 64800, -21600, -108000, -194400, -280800, -367200, -453600, -540000, -626400, -712800, -799200, -885600, -21600, 0 ] +}, { + "tz" : "America/Recife", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 170376, 83976, -2424, -88824, -175224, -261624, -348024, -434424, -520824, -607224, -693624, -780024, -866424, -2424, 0 ] +}, { + "tz" : "America/Regina", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 176316, 89916, 3516, -82884, -169284, -255684, -342084, -428484, -514884, -601284, -687684, -774084, -860484, 3516, 0 ] +}, { + "tz" : "America/Resolute", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 151200, 64800, -21600, -108000, -194400, -280800, -367200, -453600, -540000, -626400, -712800, -799200, -885600, -21600, 0 ] +}, { + "tz" : "America/Rio_Branco", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 171072, 84672, -1728, -88128, -174528, -260928, -347328, -433728, -520128, -606528, -692928, -779328, -865728, -1728, 0 ] +}, { + "tz" : "America/Rosario", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 177408, 91008, 4608, -81792, -168192, -254592, -340992, -427392, -513792, -600192, -686592, -772992, -859392, 4608, 0 ] +}, { + "tz" : "America/Santa_Isabel", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2208988800 ], + "diffs" : [ 172084, 85684, -716, -87116, -173516, -259916, -346316, -432716, -519116, -605516, -691916, -778316, -864716, -716, 0 ] +}, { + "tz" : "America/Santarem", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 175128, 88728, 2328, -84072, -170472, -256872, -343272, -429672, -516072, -602472, -688872, -775272, -861672, 2328, 0 ] +}, { + "tz" : "America/Santiago", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 175366, 88966, 2566, -83834, -170234, -256634, -343034, -429434, -515834, -602234, -688634, -775034, -861434, 2566, 0 ] +}, { + "tz" : "America/Santo_Domingo", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2524507224, -2208988800 ], + "diffs" : [ 175176, 88776, 2376, -84024, -170424, -256824, -343224, -429624, -516024, -602424, -688824, -775224, -861624, 2376, 2400, 0 ] +}, { + "tz" : "America/Sao_Paulo", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 173188, 86788, 388, -86012, -172412, -258812, -345212, -431612, -518012, -604412, -690812, -777212, -863612, 388, 0 ] +}, { + "tz" : "America/Scoresbysund", + "switches" : [ -62135766000, -59006444400, -55850684400, -52694924400, -46383404400, -43227644400, -40071884400, -33760364400, -30604604400, -27448844400, -21137324400, -17981564400, -14825804400, -12219289200, -2208988800 ], + "diffs" : [ 174472, 88072, 1672, -84728, -171128, -257528, -343928, -430328, -516728, -603128, -689528, -775928, -862328, 1672, 0 ] +}, { + "tz" : "America/Shiprock", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2717643600 ], + "diffs" : [ 172796, 86396, -4, -86404, -172804, -259204, -345604, -432004, -518404, -604804, -691204, -777604, -864004, -4, 0 ] +}, { + "tz" : "America/Sitka", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400, -3225223800, -2208988800 ], + "diffs" : [ 86473, 73, -86327, -172727, -259127, -345527, -431927, -518327, -604727, -691127, -777527, -863927, -950327, -86327, 73, 0 ] +}, { + "tz" : "America/St_Barthelemy", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/St_Johns", + "switches" : [ -62135757000, -59006435400, -55850675400, -52694915400, -46383395400, -43227635400, -40071875400, -33760355400, -30604595400, -27448835400, -21137315400, -17981555400, -14825795400, -12219280200, -2208988800 ], + "diffs" : [ 172852, 86452, 52, -86348, -172748, -259148, -345548, -431948, -518348, -604748, -691148, -777548, -863948, 52, 0 ] +}, { + "tz" : "America/St_Kitts", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/St_Lucia", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/St_Thomas", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/St_Vincent", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Swift_Current", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 177080, 90680, 4280, -82120, -168520, -254920, -341320, -427720, -514120, -600520, -686920, -773320, -859720, 4280, 0 ] +}, { + "tz" : "America/Tegucigalpa", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 172132, 85732, -668, -87068, -173468, -259868, -346268, -432668, -519068, -605468, -691868, -778268, -864668, -668, 0 ] +}, { + "tz" : "America/Thule", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 174908, 88508, 2108, -84292, -170692, -257092, -343492, -429892, -516292, -602692, -689092, -775492, -861892, 2108, 0 ] +}, { + "tz" : "America/Thunder_Bay", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2366737380, -2208988800 ], + "diffs" : [ 176220, 89820, 3420, -82980, -169380, -255780, -342180, -428580, -514980, -601380, -687780, -774180, -860580, 3420, 3600, 0 ] +}, { + "tz" : "America/Tijuana", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2208988800 ], + "diffs" : [ 172084, 85684, -716, -87116, -173516, -259916, -346316, -432716, -519116, -605516, -691916, -778316, -864716, -716, 0 ] +}, { + "tz" : "America/Toronto", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2366736148 ], + "diffs" : [ 173852, 87452, 1052, -85348, -171748, -258148, -344548, -430948, -517348, -603748, -690148, -776548, -862948, 1052, 0 ] +}, { + "tz" : "America/Tortola", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Vancouver", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2713880852 ], + "diffs" : [ 173548, 87148, 748, -85652, -172052, -258452, -344852, -431252, -517652, -604052, -690452, -776852, -863252, 748, 0 ] +}, { + "tz" : "America/Virgin", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173164, 86764, 364, -86036, -172436, -258836, -345236, -431636, -518036, -604436, -690836, -777236, -863636, 364, 0 ] +}, { + "tz" : "America/Whitehorse", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2208988800 ], + "diffs" : [ 176412, 90012, 3612, -82788, -169188, -255588, -341988, -428388, -514788, -601188, -687588, -773988, -860388, 3612, 0 ] +}, { + "tz" : "America/Winnipeg", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2602258284 ], + "diffs" : [ 174516, 88116, 1716, -84684, -171084, -257484, -343884, -430284, -516684, -603084, -689484, -775884, -862284, 1716, 0 ] +}, { + "tz" : "America/Yakutat", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400, -3225224862, -2208988800 ], + "diffs" : [ 87535, 1135, -85265, -171665, -258065, -344465, -430865, -517265, -603665, -690065, -776465, -862865, -949265, -85265, 1135, 0 ] +}, { + "tz" : "America/Yellowknife", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 147600, 61200, -25200, -111600, -198000, -284400, -370800, -457200, -543600, -630000, -716400, -802800, -889200, -25200, 0 ] +}, { + "tz" : "Antarctica/Casey", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 201600, 115200, 28800, -57600, -144000, -230400, -316800, -403200, -489600, -576000, -662400, -748800, -835200, 28800, 0 ] +}, { + "tz" : "Antarctica/Davis", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 198000, 111600, 25200, -61200, -147600, -234000, -320400, -406800, -493200, -579600, -666000, -752400, -838800, 25200, 0 ] +}, { + "tz" : "Antarctica/DumontDUrville", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2208988800 ], + "diffs" : [ 208800, 122400, 36000, -50400, -136800, -223200, -309600, -396000, -482400, -568800, -655200, -741600, -828000, 36000, 0 ] +}, { + "tz" : "Antarctica/Macquarie", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -2214262800, -2208988800 ], + "diffs" : [ 212400, 126000, 39600, -46800, -133200, -219600, -306000, -392400, -478800, -565200, -651600, -738000, -824400, 39600, 3600, 0 ] +}, { + "tz" : "Antarctica/Mawson", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 190800, 104400, 18000, -68400, -154800, -241200, -327600, -414000, -500400, -586800, -673200, -759600, -846000, 18000, 0 ] +}, { + "tz" : "Antarctica/McMurdo", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -3192437344, -2208988800 ], + "diffs" : [ 174056, 87656, 1256, -85144, -171544, -257944, -344344, -430744, -517144, -603544, -689944, -776344, -862744, 1256, 1800, 0 ] +}, { + "tz" : "Antarctica/Palmer", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 162000, 75600, -10800, -97200, -183600, -270000, -356400, -442800, -529200, -615600, -702000, -788400, -874800, -10800, 0 ] +}, { + "tz" : "Antarctica/Rothera", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 162000, 75600, -10800, -97200, -183600, -270000, -356400, -442800, -529200, -615600, -702000, -788400, -874800, -10800, 0 ] +}, { + "tz" : "Antarctica/South_Pole", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -3192437344, -2208988800 ], + "diffs" : [ 174056, 87656, 1256, -85144, -171544, -257944, -344344, -430744, -517144, -603544, -689944, -776344, -862744, 1256, 1800, 0 ] +}, { + "tz" : "Antarctica/Syowa", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 183600, 97200, 10800, -75600, -162000, -248400, -334800, -421200, -507600, -594000, -680400, -766800, -853200, 10800, 0 ] +}, { + "tz" : "Antarctica/Troll", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Antarctica/Vostok", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 194400, 108000, 21600, -64800, -151200, -237600, -324000, -410400, -496800, -583200, -669600, -756000, -842400, 21600, 0 ] +}, { + "tz" : "Arctic/Longyearbyen", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2366757780 ], + "diffs" : [ 173820, 87420, 1020, -85380, -171780, -258180, -344580, -430980, -517380, -603780, -690180, -776580, -862980, 1020, 0 ] +}, { + "tz" : "Asia/Aden", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 172388, 85988, -412, -86812, -173212, -259612, -346012, -432412, -518812, -605212, -691612, -778012, -864412, -412, 0 ] +}, { + "tz" : "Asia/Almaty", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 175932, 89532, 3132, -83268, -169668, -256068, -342468, -428868, -515268, -601668, -688068, -774468, -860868, 3132, 0 ] +}, { + "tz" : "Asia/Amman", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 171376, 84976, -1424, -87824, -174224, -260624, -347024, -433424, -519824, -606224, -692624, -779024, -865424, -1424, 0 ] +}, { + "tz" : "Asia/Anadyr", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 173404, 87004, 604, -85796, -172196, -258596, -344996, -431396, -517796, -604196, -690596, -776996, -863396, 604, 0 ] +}, { + "tz" : "Asia/Aqtau", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 178736, 92336, 5936, -80464, -166864, -253264, -339664, -426064, -512464, -598864, -685264, -771664, -858064, 5936, 0 ] +}, { + "tz" : "Asia/Aqtobe", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 177080, 90680, 4280, -82120, -168520, -254920, -341320, -427720, -514120, -600520, -686920, -773320, -859720, 4280, 0 ] +}, { + "tz" : "Asia/Ashgabat", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 176788, 90388, 3988, -82412, -168812, -255212, -341612, -428012, -514412, -600812, -687212, -773612, -860012, 3988, 0 ] +}, { + "tz" : "Asia/Ashkhabad", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 176788, 90388, 3988, -82412, -168812, -255212, -341612, -428012, -514412, -600812, -687212, -773612, -860012, 3988, 0 ] +}, { + "tz" : "Asia/Atyrau", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 178336, 91936, 5536, -80864, -167264, -253664, -340064, -426464, -512864, -599264, -685664, -772064, -858464, 5536, 0 ] +}, { + "tz" : "Asia/Baghdad", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2524532404, -2208988800 ], + "diffs" : [ 172940, 86540, 140, -86260, -172660, -259060, -345460, -431860, -518260, -604660, -691060, -777460, -863860, 140, 144, 0 ] +}, { + "tz" : "Asia/Bahrain", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 171232, 84832, -1568, -87968, -174368, -260768, -347168, -433568, -519968, -606368, -692768, -779168, -865568, -1568, 0 ] +}, { + "tz" : "Asia/Baku", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 175236, 88836, 2436, -83964, -170364, -256764, -343164, -429564, -515964, -602364, -688764, -775164, -861564, 2436, 0 ] +}, { + "tz" : "Asia/Bangkok", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 173876, 87476, 1076, -85324, -171724, -258124, -344524, -430924, -517324, -603724, -690124, -776524, -862924, 1076, 0 ] +}, { + "tz" : "Asia/Barnaul", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 177900, 91500, 5100, -81300, -167700, -254100, -340500, -426900, -513300, -599700, -686100, -772500, -858900, 5100, 0 ] +}, { + "tz" : "Asia/Beirut", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2840149320 ], + "diffs" : [ 171480, 85080, -1320, -87720, -174120, -260520, -346920, -433320, -519720, -606120, -692520, -778920, -865320, -1320, 0 ] +}, { + "tz" : "Asia/Bishkek", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 176496, 90096, 3696, -82704, -169104, -255504, -341904, -428304, -514704, -601104, -687504, -773904, -860304, 3696, 0 ] +}, { + "tz" : "Asia/Brunei", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 174020, 87620, 1220, -85180, -171580, -257980, -344380, -430780, -517180, -603580, -689980, -776380, -862780, 1220, 0 ] +}, { + "tz" : "Asia/Calcutta", + "switches" : [ -62135789400, -59006467800, -55850707800, -52694947800, -46383427800, -43227667800, -40071907800, -33760387800, -30604627800, -27448867800, -21137347800, -17981587800, -14825827800, -12219312600, -3645235808, -3155695330, -2208988800 ], + "diffs" : [ 171392, 84992, -1408, -87808, -174208, -260608, -347008, -433408, -519808, -606208, -692608, -779008, -865408, -1408, -1400, 530, 0 ] +}, { + "tz" : "Asia/Chita", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2208988800 ], + "diffs" : [ 177968, 91568, 5168, -81232, -167632, -254032, -340432, -426832, -513232, -599632, -686032, -772432, -858832, 5168, 0 ] +}, { + "tz" : "Asia/Choibalsan", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 174120, 87720, 1320, -85080, -171480, -257880, -344280, -430680, -517080, -603480, -689880, -776280, -862680, 1320, 0 ] +}, { + "tz" : "Asia/Chongqing", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 172457, 86057, -343, -86743, -173143, -259543, -345943, -432343, -518743, -605143, -691543, -777943, -864343, -343, 0 ] +}, { + "tz" : "Asia/Chungking", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 172457, 86057, -343, -86743, -173143, -259543, -345943, -432343, -518743, -605143, -691543, -777943, -864343, -343, 0 ] +}, { + "tz" : "Asia/Colombo", + "switches" : [ -62135789400, -59006467800, -55850707800, -52694947800, -46383427800, -43227667800, -40071907800, -33760387800, -30604627800, -27448867800, -21137347800, -17981587800, -14825827800, -12219312600, -2840160592, -2208988800 ], + "diffs" : [ 173436, 87036, 636, -85764, -172164, -258564, -344964, -431364, -517764, -604164, -690564, -776964, -863364, 636, 628, 0 ] +}, { + "tz" : "Asia/Dacca", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2524543700, -2208988800 ], + "diffs" : [ 172700, 86300, -100, -86500, -172900, -259300, -345700, -432100, -518500, -604900, -691300, -777700, -864100, -100, 400, 0 ] +}, { + "tz" : "Asia/Damascus", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 171288, 84888, -1512, -87912, -174312, -260712, -347112, -433512, -519912, -606312, -692712, -779112, -865512, -1512, 0 ] +}, { + "tz" : "Asia/Dhaka", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2524543700, -2208988800 ], + "diffs" : [ 172700, 86300, -100, -86500, -172900, -259300, -345700, -432100, -518500, -604900, -691300, -777700, -864100, -100, 400, 0 ] +}, { + "tz" : "Asia/Dili", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2208988800 ], + "diffs" : [ 175060, 88660, 2260, -84140, -170540, -256940, -343340, -429740, -516140, -602540, -688940, -775340, -861740, 2260, 0 ] +}, { + "tz" : "Asia/Dubai", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 173928, 87528, 1128, -85272, -171672, -258072, -344472, -430872, -517272, -603672, -690072, -776472, -862872, 1128, 0 ] +}, { + "tz" : "Asia/Dushanbe", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 174288, 87888, 1488, -84912, -171312, -257712, -344112, -430512, -516912, -603312, -689712, -776112, -862512, 1488, 0 ] +}, { + "tz" : "Asia/Famagusta", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 171852, 85452, -948, -87348, -173748, -260148, -346548, -432948, -519348, -605748, -692148, -778548, -864948, -948, 0 ] +}, { + "tz" : "Asia/Gaza", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 171728, 85328, -1072, -87472, -173872, -260272, -346672, -433072, -519472, -605872, -692272, -778672, -865072, -1072, 0 ] +}, { + "tz" : "Asia/Harbin", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 172457, 86057, -343, -86743, -173143, -259543, -345943, -432343, -518743, -605143, -691543, -777943, -864343, -343, 0 ] +}, { + "tz" : "Asia/Hebron", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 171577, 85177, -1223, -87623, -174023, -260423, -346823, -433223, -519623, -606023, -692423, -778823, -865223, -1223, 0 ] +}, { + "tz" : "Asia/Ho_Chi_Minh", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 172400, 86000, -400, -86800, -173200, -259600, -346000, -432400, -518800, -605200, -691600, -778000, -864400, -400, 0 ] +}, { + "tz" : "Asia/Hong_Kong", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 174198, 87798, 1398, -85002, -171402, -257802, -344202, -430602, -517002, -603402, -689802, -776202, -862602, 1398, 0 ] +}, { + "tz" : "Asia/Hovd", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 176004, 89604, 3204, -83196, -169596, -255996, -342396, -428796, -515196, -601596, -687996, -774396, -860796, 3204, 0 ] +}, { + "tz" : "Asia/Irkutsk", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 176575, 90175, 3775, -82625, -169025, -255425, -341825, -428225, -514625, -601025, -687425, -773825, -860225, 3775, 0 ] +}, { + "tz" : "Asia/Istanbul", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2840151536, -2208988800 ], + "diffs" : [ 176648, 90248, 3848, -82552, -168952, -255352, -341752, -428152, -514552, -600952, -687352, -773752, -860152, 3848, 3784, 0 ] +}, { + "tz" : "Asia/Jakarta", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 172368, 85968, -432, -86832, -173232, -259632, -346032, -432432, -518832, -605232, -691632, -778032, -864432, -432, 0 ] +}, { + "tz" : "Asia/Jayapura", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2208988800 ], + "diffs" : [ 171432, 85032, -1368, -87768, -174168, -260568, -346968, -433368, -519768, -606168, -692568, -778968, -865368, -1368, 0 ] +}, { + "tz" : "Asia/Jerusalem", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2840148014, -2208988800 ], + "diffs" : [ 171546, 85146, -1254, -87654, -174054, -260454, -346854, -433254, -519654, -606054, -692454, -778854, -865254, -1254, -1240, 0 ] +}, { + "tz" : "Asia/Kabul", + "switches" : [ -62135785800, -59006464200, -55850704200, -52694944200, -46383424200, -43227664200, -40071904200, -33760384200, -30604624200, -27448864200, -21137344200, -17981584200, -14825824200, -12219309000, -2524540008, -2208988800 ], + "diffs" : [ 172392, 85992, -408, -86808, -173208, -259608, -346008, -432408, -518808, -605208, -691608, -778008, -864408, -408, 1800, 0 ] +}, { + "tz" : "Asia/Kamchatka", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 177924, 91524, 5124, -81276, -167676, -254076, -340476, -426876, -513276, -599676, -686076, -772476, -858876, 5124, 0 ] +}, { + "tz" : "Asia/Karachi", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 174708, 88308, 1908, -84492, -170892, -257292, -343692, -430092, -516492, -602892, -689292, -775692, -862092, 1908, 0 ] +}, { + "tz" : "Asia/Kashgar", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 173380, 86980, 580, -85820, -172220, -258620, -345020, -431420, -517820, -604220, -690620, -777020, -863420, 580, 0 ] +}, { + "tz" : "Asia/Kathmandu", + "switches" : [ -62135790300, -59006468700, -55850708700, -52694948700, -46383428700, -43227668700, -40071908700, -33760388700, -30604628700, -27448868700, -21137348700, -17981588700, -14825828700, -12219313500, -2208988800 ], + "diffs" : [ 173024, 86624, 224, -86176, -172576, -258976, -345376, -431776, -518176, -604576, -690976, -777376, -863776, 224, 0 ] +}, { + "tz" : "Asia/Katmandu", + "switches" : [ -62135790300, -59006468700, -55850708700, -52694948700, -46383428700, -43227668700, -40071908700, -33760388700, -30604628700, -27448868700, -21137348700, -17981588700, -14825828700, -12219313500, -2208988800 ], + "diffs" : [ 173024, 86624, 224, -86176, -172576, -258976, -345376, -431776, -518176, -604576, -690976, -777376, -863776, 224, 0 ] +}, { + "tz" : "Asia/Khandyga", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2208988800 ], + "diffs" : [ 172667, 86267, -133, -86533, -172933, -259333, -345733, -432133, -518533, -604933, -691333, -777733, -864133, -133, 0 ] +}, { + "tz" : "Asia/Kolkata", + "switches" : [ -62135789400, -59006467800, -55850707800, -52694947800, -46383427800, -43227667800, -40071907800, -33760387800, -30604627800, -27448867800, -21137347800, -17981587800, -14825827800, -12219312600, -3645235808, -3155695330, -2208988800 ], + "diffs" : [ 171392, 84992, -1408, -87808, -174208, -260608, -347008, -433408, -519808, -606208, -692608, -779008, -865408, -1408, -1400, 530, 0 ] +}, { + "tz" : "Asia/Krasnoyarsk", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 175714, 89314, 2914, -83486, -169886, -256286, -342686, -429086, -515486, -601886, -688286, -774686, -861086, 2914, 0 ] +}, { + "tz" : "Asia/Kuala_Lumpur", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 177194, 90794, 4394, -82006, -168406, -254806, -341206, -427606, -514006, -600406, -686806, -773206, -859606, 4394, 0 ] +}, { + "tz" : "Asia/Kuching", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 175120, 88720, 2320, -84080, -170480, -256880, -343280, -429680, -516080, -602480, -688880, -775280, -861680, 2320, 0 ] +}, { + "tz" : "Asia/Kuwait", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 172388, 85988, -412, -86812, -173212, -259612, -346012, -432412, -518812, -605212, -691612, -778012, -864412, -412, 0 ] +}, { + "tz" : "Asia/Macao", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 174350, 87950, 1550, -84850, -171250, -257650, -344050, -430450, -516850, -603250, -689650, -776050, -862450, 1550, 0 ] +}, { + "tz" : "Asia/Macau", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 174350, 87950, 1550, -84850, -171250, -257650, -344050, -430450, -516850, -603250, -689650, -776050, -862450, 1550, 0 ] +}, { + "tz" : "Asia/Magadan", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -2208988800 ], + "diffs" : [ 176208, 89808, 3408, -82992, -169392, -255792, -342192, -428592, -514992, -601392, -687792, -774192, -860592, 3408, 0 ] +}, { + "tz" : "Asia/Makassar", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 172944, 86544, 144, -86256, -172656, -259056, -345456, -431856, -518256, -604656, -691056, -777456, -863856, 144, 0 ] +}, { + "tz" : "Asia/Manila", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -3944620800, -2229321840 ], + "diffs" : [ 258960, 172560, 86160, -240, -86640, -173040, -259440, -345840, -432240, -518640, -605040, -691440, -777840, 86160, -240, 0 ] +}, { + "tz" : "Asia/Muscat", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 173928, 87528, 1128, -85272, -171672, -258072, -344472, -430872, -517272, -603672, -690072, -776472, -862872, 1128, 0 ] +}, { + "tz" : "Asia/Nicosia", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 171992, 85592, -808, -87208, -173608, -260008, -346408, -432808, -519208, -605608, -692008, -778408, -864808, -808, 0 ] +}, { + "tz" : "Asia/Novokuznetsk", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 177072, 90672, 4272, -82128, -168528, -254928, -341328, -427728, -514128, -600528, -686928, -773328, -859728, 4272, 0 ] +}, { + "tz" : "Asia/Novosibirsk", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 178100, 91700, 5300, -81100, -167500, -253900, -340300, -426700, -513100, -599500, -685900, -772300, -858700, 5300, 0 ] +}, { + "tz" : "Asia/Omsk", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 176790, 90390, 3990, -82410, -168810, -255210, -341610, -428010, -514410, -600810, -687210, -773610, -860010, 3990, 0 ] +}, { + "tz" : "Asia/Oral", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 178476, 92076, 5676, -80724, -167124, -253524, -339924, -426324, -512724, -599124, -685524, -771924, -858324, 5676, 0 ] +}, { + "tz" : "Asia/Phnom_Penh", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 173876, 87476, 1076, -85324, -171724, -258124, -344524, -430924, -517324, -603724, -690124, -776524, -862924, 1076, 0 ] +}, { + "tz" : "Asia/Pontianak", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 171760, 85360, -1040, -87440, -173840, -260240, -346640, -433040, -519440, -605840, -692240, -778640, -865040, -1040, 0 ] +}, { + "tz" : "Asia/Pyongyang", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2208988800 ], + "diffs" : [ 175020, 88620, 2220, -84180, -170580, -256980, -343380, -429780, -516180, -602580, -688980, -775380, -861780, 2220, 0 ] +}, { + "tz" : "Asia/Qatar", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 171232, 84832, -1568, -87968, -174368, -260768, -347168, -433568, -519968, -606368, -692768, -779168, -865568, -1568, 0 ] +}, { + "tz" : "Asia/Qostanay", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 179132, 92732, 6332, -80068, -166468, -252868, -339268, -425668, -512068, -598468, -684868, -771268, -857668, 6332, 0 ] +}, { + "tz" : "Asia/Qyzylorda", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 175088, 88688, 2288, -84112, -170512, -256912, -343312, -429712, -516112, -602512, -688912, -775312, -861712, 2288, 0 ] +}, { + "tz" : "Asia/Rangoon", + "switches" : [ -62135793000, -59006471400, -55850711400, -52694951400, -46383431400, -43227671400, -40071911400, -33760391400, -30604631400, -27448871400, -21137351400, -17981591400, -14825831400, -12219316200, -2208988800 ], + "diffs" : [ 173113, 86713, 313, -86087, -172487, -258887, -345287, -431687, -518087, -604487, -690887, -777287, -863687, 313, 0 ] +}, { + "tz" : "Asia/Riyadh", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 172388, 85988, -412, -86812, -173212, -259612, -346012, -432412, -518812, -605212, -691612, -778012, -864412, -412, 0 ] +}, { + "tz" : "Asia/Saigon", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 172400, 86000, -400, -86800, -173200, -259600, -346000, -432400, -518800, -605200, -691600, -778000, -864400, -400, 0 ] +}, { + "tz" : "Asia/Sakhalin", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -2208988800 ], + "diffs" : [ 178152, 91752, 5352, -81048, -167448, -253848, -340248, -426648, -513048, -599448, -685848, -772248, -858648, 5352, 0 ] +}, { + "tz" : "Asia/Samarkand", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 174727, 88327, 1927, -84473, -170873, -257273, -343673, -430073, -516473, -602873, -689273, -775673, -862073, 1927, 0 ] +}, { + "tz" : "Asia/Seoul", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2208988800 ], + "diffs" : [ 174728, 88328, 1928, -84472, -170872, -257272, -343672, -430072, -516472, -602872, -689272, -775672, -862072, 1928, 0 ] +}, { + "tz" : "Asia/Shanghai", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 172457, 86057, -343, -86743, -173143, -259543, -345943, -432343, -518743, -605143, -691543, -777943, -864343, -343, 0 ] +}, { + "tz" : "Asia/Singapore", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 176675, 90275, 3875, -82525, -168925, -255325, -341725, -428125, -514525, -600925, -687325, -773725, -860125, 3875, 0 ] +}, { + "tz" : "Asia/Srednekolymsk", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -2208988800 ], + "diffs" : [ 175508, 89108, 2708, -83692, -170092, -256492, -342892, -429292, -515692, -602092, -688492, -774892, -861292, 2708, 0 ] +}, { + "tz" : "Asia/Taipei", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2335248360 ], + "diffs" : [ 172440, 86040, -360, -86760, -173160, -259560, -345960, -432360, -518760, -605160, -691560, -777960, -864360, -360, 0 ] +}, { + "tz" : "Asia/Tashkent", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 174169, 87769, 1369, -85031, -171431, -257831, -344231, -430631, -517031, -603431, -689831, -776231, -862631, 1369, 0 ] +}, { + "tz" : "Asia/Tbilisi", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 176449, 90049, 3649, -82751, -169151, -255551, -341951, -428351, -514751, -601151, -687551, -773951, -860351, 3649, 0 ] +}, { + "tz" : "Asia/Tel_Aviv", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2840148014, -2208988800 ], + "diffs" : [ 171546, 85146, -1254, -87654, -174054, -260454, -346854, -433254, -519654, -606054, -692454, -778854, -865254, -1254, -1240, 0 ] +}, { + "tz" : "Asia/Thimbu", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 172884, 86484, 84, -86316, -172716, -259116, -345516, -431916, -518316, -604716, -691116, -777516, -863916, 84, 0 ] +}, { + "tz" : "Asia/Thimphu", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 172884, 86484, 84, -86316, -172716, -259116, -345516, -431916, -518316, -604716, -691116, -777516, -863916, 84, 0 ] +}, { + "tz" : "Asia/Tokyo", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2587712400 ], + "diffs" : [ 171661, 85261, -1139, -87539, -173939, -260339, -346739, -433139, -519539, -605939, -692339, -778739, -865139, -1139, 0 ] +}, { + "tz" : "Asia/Tomsk", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 177609, 91209, 4809, -81591, -167991, -254391, -340791, -427191, -513591, -599991, -686391, -772791, -859191, 4809, 0 ] +}, { + "tz" : "Asia/Ujung_Pandang", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 172944, 86544, 144, -86256, -172656, -259056, -345456, -431856, -518256, -604656, -691056, -777456, -863856, 144, 0 ] +}, { + "tz" : "Asia/Ulaanbaatar", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 175948, 89548, 3148, -83252, -169652, -256052, -342452, -428852, -515252, -601652, -688052, -774452, -860852, 3148, 0 ] +}, { + "tz" : "Asia/Ulan_Bator", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 175948, 89548, 3148, -83252, -169652, -256052, -342452, -428852, -515252, -601652, -688052, -774452, -860852, 3148, 0 ] +}, { + "tz" : "Asia/Urumqi", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 173380, 86980, 580, -85820, -172220, -258620, -345020, -431420, -517820, -604220, -690620, -777020, -863420, 580, 0 ] +}, { + "tz" : "Asia/Ust-Nera", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2208988800 ], + "diffs" : [ 174426, 88026, 1626, -84774, -171174, -257574, -343974, -430374, -516774, -603174, -689574, -775974, -862374, 1626, 0 ] +}, { + "tz" : "Asia/Vientiane", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2208988800 ], + "diffs" : [ 173876, 87476, 1076, -85324, -171724, -258124, -344524, -430924, -517324, -603724, -690124, -776524, -862924, 1076, 0 ] +}, { + "tz" : "Asia/Vladivostok", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2208988800 ], + "diffs" : [ 177149, 90749, 4349, -82051, -168451, -254851, -341251, -427651, -514051, -600451, -686851, -773251, -859651, 4349, 0 ] +}, { + "tz" : "Asia/Yakutsk", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2208988800 ], + "diffs" : [ 174062, 87662, 1262, -85138, -171538, -257938, -344338, -430738, -517138, -603538, -689938, -776338, -862738, 1262, 0 ] +}, { + "tz" : "Asia/Yangon", + "switches" : [ -62135793000, -59006471400, -55850711400, -52694951400, -46383431400, -43227671400, -40071911400, -33760391400, -30604631400, -27448871400, -21137351400, -17981591400, -14825831400, -12219316200, -2208988800 ], + "diffs" : [ 173113, 86713, 313, -86087, -172487, -258887, -345287, -431687, -518087, -604487, -690887, -777287, -863687, 313, 0 ] +}, { + "tz" : "Asia/Yekaterinburg", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 176247, 89847, 3447, -82953, -169353, -255753, -342153, -428553, -514953, -601353, -687753, -774153, -860553, 3447, 0 ] +}, { + "tz" : "Asia/Yerevan", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 176520, 90120, 3720, -82680, -169080, -255480, -341880, -428280, -514680, -601080, -687480, -773880, -860280, 3720, 0 ] +}, { + "tz" : "Atlantic/Azores", + "switches" : [ -62135766000, -59006444400, -55850684400, -52694924400, -46383404400, -43227644400, -40071884400, -33760364400, -30604604400, -27448844400, -21137324400, -17981564400, -14825804400, -12219289200, -2713907512, -2208988800 ], + "diffs" : [ 175360, 88960, 2560, -83840, -170240, -256640, -343040, -429440, -515840, -602240, -688640, -775040, -861440, 2560, 3272, 0 ] +}, { + "tz" : "Atlantic/Bermuda", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173958, 87558, 1158, -85242, -171642, -258042, -344442, -430842, -517242, -603642, -690042, -776442, -862842, 1158, 0 ] +}, { + "tz" : "Atlantic/Canary", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 176496, 90096, 3696, -82704, -169104, -255504, -341904, -428304, -514704, -601104, -687504, -773904, -860304, 3696, 0 ] +}, { + "tz" : "Atlantic/Cape_Verde", + "switches" : [ -62135766000, -59006444400, -55850684400, -52694924400, -46383404400, -43227644400, -40071884400, -33760364400, -30604604400, -27448844400, -21137324400, -17981564400, -14825804400, -12219289200, -2208988800 ], + "diffs" : [ 174844, 88444, 2044, -84356, -170756, -257156, -343556, -429956, -516356, -602756, -689156, -775556, -861956, 2044, 0 ] +}, { + "tz" : "Atlantic/Faeroe", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 174424, 88024, 1624, -84776, -171176, -257576, -343976, -430376, -516776, -603176, -689576, -775976, -862376, 1624, 0 ] +}, { + "tz" : "Atlantic/Faroe", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 174424, 88024, 1624, -84776, -171176, -257576, -343976, -430376, -516776, -603176, -689576, -775976, -862376, 1624, 0 ] +}, { + "tz" : "Atlantic/Jan_Mayen", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2366757780 ], + "diffs" : [ 173820, 87420, 1020, -85380, -171780, -258180, -344580, -430980, -517380, -603780, -690180, -776580, -862980, 1020, 0 ] +}, { + "tz" : "Atlantic/Madeira", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 176856, 90456, 4056, -82344, -168744, -255144, -341544, -427944, -514344, -600744, -687144, -773544, -859944, 4056, 0 ] +}, { + "tz" : "Atlantic/Reykjavik", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 178080, 91680, 5280, -81120, -167520, -253920, -340320, -426720, -513120, -599520, -685920, -772320, -858720, 5280, 0 ] +}, { + "tz" : "Atlantic/South_Georgia", + "switches" : [ -62135762400, -59006440800, -55850680800, -52694920800, -46383400800, -43227640800, -40071880800, -33760360800, -30604600800, -27448840800, -21137320800, -17981560800, -14825800800, -12219285600, -2524512832 ], + "diffs" : [ 174368, 87968, 1568, -84832, -171232, -257632, -344032, -430432, -516832, -603232, -689632, -776032, -862432, 1568, 0 ] +}, { + "tz" : "Atlantic/St_Helena", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 173768, 87368, 968, -85432, -171832, -258232, -344632, -431032, -517432, -603832, -690232, -776632, -863032, 968, 0 ] +}, { + "tz" : "Atlantic/Stanley", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 175884, 89484, 3084, -83316, -169716, -256116, -342516, -428916, -515316, -601716, -688116, -774516, -860916, 3084, 0 ] +}, { + "tz" : "Australia/ACT", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2364113092 ], + "diffs" : [ 172508, 86108, -292, -86692, -173092, -259492, -345892, -432292, -518692, -605092, -691492, -777892, -864292, -292, 0 ] +}, { + "tz" : "Australia/Adelaide", + "switches" : [ -62135803800, -59006482200, -55850722200, -52694962200, -46383442200, -43227682200, -40071922200, -33760402200, -30604642200, -27448882200, -21137362200, -17981602200, -14825842200, -12219327000, -2364111860, -2230189200 ], + "diffs" : [ 173740, 87340, 940, -85460, -171860, -258260, -344660, -431060, -517460, -603860, -690260, -776660, -863060, 940, 1800, 0 ] +}, { + "tz" : "Australia/Brisbane", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2366791928 ], + "diffs" : [ 172072, 85672, -728, -87128, -173528, -259928, -346328, -432728, -519128, -605528, -691928, -778328, -864728, -728, 0 ] +}, { + "tz" : "Australia/Broken_Hill", + "switches" : [ -62135803800, -59006482200, -55850722200, -52694962200, -46383442200, -43227682200, -40071922200, -33760402200, -30604642200, -27448882200, -21137362200, -17981602200, -14825842200, -12219327000, -2364108948, -2314953000, -2230189200 ], + "diffs" : [ 173052, 86652, 252, -86148, -172548, -258948, -345348, -431748, -518148, -604548, -690948, -777348, -863748, 252, -1800, 1800, 0 ] +}, { + "tz" : "Australia/Canberra", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2364113092 ], + "diffs" : [ 172508, 86108, -292, -86692, -173092, -259492, -345892, -432292, -518692, -605092, -691492, -777892, -864292, -292, 0 ] +}, { + "tz" : "Australia/Currie", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2345794528 ], + "diffs" : [ 174272, 87872, 1472, -84928, -171328, -257728, -344128, -430528, -516928, -603328, -689728, -776128, -862528, 1472, 0 ] +}, { + "tz" : "Australia/Darwin", + "switches" : [ -62135803800, -59006482200, -55850722200, -52694962200, -46383442200, -43227682200, -40071922200, -33760402200, -30604642200, -27448882200, -21137362200, -17981602200, -14825842200, -12219327000, -2364110000, -2230189200 ], + "diffs" : [ 175600, 89200, 2800, -83600, -170000, -256400, -342800, -429200, -515600, -602000, -688400, -774800, -861200, 2800, 1800, 0 ] +}, { + "tz" : "Australia/Eucla", + "switches" : [ -62135801100, -59006479500, -55850719500, -52694959500, -46383439500, -43227679500, -40071919500, -33760399500, -30604639500, -27448879500, -21137359500, -17981599500, -14825839500, -12219324300, -2337928528 ], + "diffs" : [ 173372, 86972, 572, -85828, -172228, -258628, -345028, -431428, -517828, -604228, -690628, -777028, -863428, 572, 0 ] +}, { + "tz" : "Australia/Hobart", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2345795356 ], + "diffs" : [ 173444, 87044, 644, -85756, -172156, -258556, -344956, -431356, -517756, -604156, -690556, -776956, -863356, 644, 0 ] +}, { + "tz" : "Australia/LHI", + "switches" : [ -62135807400, -59006485800, -55850725800, -52694965800, -46383445800, -43227685800, -40071925800, -33760405800, -30604645800, -27448885800, -21137365800, -17981605800, -14825845800, -12219330600, -2364116780, -2208988800 ], + "diffs" : [ 172420, 86020, -380, -86780, -173180, -259580, -345980, -432380, -518780, -605180, -691580, -777980, -864380, -380, 1800, 0 ] +}, { + "tz" : "Australia/Lindeman", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2366790956 ], + "diffs" : [ 173044, 86644, 244, -86156, -172556, -258956, -345356, -431756, -518156, -604556, -690956, -777356, -863756, 244, 0 ] +}, { + "tz" : "Australia/Lord_Howe", + "switches" : [ -62135807400, -59006485800, -55850725800, -52694965800, -46383445800, -43227685800, -40071925800, -33760405800, -30604645800, -27448885800, -21137365800, -17981605800, -14825845800, -12219330600, -2364116780, -2208988800 ], + "diffs" : [ 172420, 86020, -380, -86780, -173180, -259580, -345980, -432380, -518780, -605180, -691580, -777980, -864380, -380, 1800, 0 ] +}, { + "tz" : "Australia/Melbourne", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2364111592 ], + "diffs" : [ 174008, 87608, 1208, -85192, -171592, -257992, -344392, -430792, -517192, -603592, -689992, -776392, -862792, 1208, 0 ] +}, { + "tz" : "Australia/NSW", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2364113092 ], + "diffs" : [ 172508, 86108, -292, -86692, -173092, -259492, -345892, -432292, -518692, -605092, -691492, -777892, -864292, -292, 0 ] +}, { + "tz" : "Australia/North", + "switches" : [ -62135803800, -59006482200, -55850722200, -52694962200, -46383442200, -43227682200, -40071922200, -33760402200, -30604642200, -27448882200, -21137362200, -17981602200, -14825842200, -12219327000, -2364110000, -2230189200 ], + "diffs" : [ 175600, 89200, 2800, -83600, -170000, -256400, -342800, -429200, -515600, -602000, -688400, -774800, -861200, 2800, 1800, 0 ] +}, { + "tz" : "Australia/Perth", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2337925404 ], + "diffs" : [ 173796, 87396, 996, -85404, -171804, -258204, -344604, -431004, -517404, -603804, -690204, -776604, -863004, 996, 0 ] +}, { + "tz" : "Australia/Queensland", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2366791928 ], + "diffs" : [ 172072, 85672, -728, -87128, -173528, -259928, -346328, -432728, -519128, -605528, -691928, -778328, -864728, -728, 0 ] +}, { + "tz" : "Australia/South", + "switches" : [ -62135803800, -59006482200, -55850722200, -52694962200, -46383442200, -43227682200, -40071922200, -33760402200, -30604642200, -27448882200, -21137362200, -17981602200, -14825842200, -12219327000, -2364111860, -2230189200 ], + "diffs" : [ 173740, 87340, 940, -85460, -171860, -258260, -344660, -431060, -517460, -603860, -690260, -776660, -863060, 940, 1800, 0 ] +}, { + "tz" : "Australia/Sydney", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2364113092 ], + "diffs" : [ 172508, 86108, -292, -86692, -173092, -259492, -345892, -432292, -518692, -605092, -691492, -777892, -864292, -292, 0 ] +}, { + "tz" : "Australia/Tasmania", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2345795356 ], + "diffs" : [ 173444, 87044, 644, -85756, -172156, -258556, -344956, -431356, -517756, -604156, -690556, -776956, -863356, 644, 0 ] +}, { + "tz" : "Australia/Victoria", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2364111592 ], + "diffs" : [ 174008, 87608, 1208, -85192, -171592, -257992, -344392, -430792, -517192, -603592, -689992, -776392, -862792, 1208, 0 ] +}, { + "tz" : "Australia/West", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2337925404 ], + "diffs" : [ 173796, 87396, 996, -85404, -171804, -258204, -344604, -431004, -517404, -603804, -690204, -776604, -863004, 996, 0 ] +}, { + "tz" : "Australia/Yancowinna", + "switches" : [ -62135803800, -59006482200, -55850722200, -52694962200, -46383442200, -43227682200, -40071922200, -33760402200, -30604642200, -27448882200, -21137362200, -17981602200, -14825842200, -12219327000, -2364108948, -2314953000, -2230189200 ], + "diffs" : [ 173052, 86652, 252, -86148, -172548, -258948, -345348, -431748, -518148, -604548, -690948, -777348, -863748, 252, -1800, 1800, 0 ] +}, { + "tz" : "Brazil/Acre", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 171072, 84672, -1728, -88128, -174528, -260928, -347328, -433728, -520128, -606528, -692928, -779328, -865728, -1728, 0 ] +}, { + "tz" : "Brazil/DeNoronha", + "switches" : [ -62135762400, -59006440800, -55850680800, -52694920800, -46383400800, -43227640800, -40071880800, -33760360800, -30604600800, -27448840800, -21137320800, -17981560800, -14825800800, -12219285600, -2208988800 ], + "diffs" : [ 173380, 86980, 580, -85820, -172220, -258620, -345020, -431420, -517820, -604220, -690620, -777020, -863420, 580, 0 ] +}, { + "tz" : "Brazil/East", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000, -2208988800 ], + "diffs" : [ 173188, 86788, 388, -86012, -172412, -258812, -345212, -431612, -518012, -604412, -690812, -777212, -863612, 388, 0 ] +}, { + "tz" : "Brazil/West", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 172804, 86404, 4, -86396, -172796, -259196, -345596, -431996, -518396, -604796, -691196, -777596, -863996, 4, 0 ] +}, { + "tz" : "CET", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "CST6CDT", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Canada/Atlantic", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 173664, 87264, 864, -85536, -171936, -258336, -344736, -431136, -517536, -603936, -690336, -776736, -863136, 864, 0 ] +}, { + "tz" : "Canada/Central", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2602258284 ], + "diffs" : [ 174516, 88116, 1716, -84684, -171084, -257484, -343884, -430284, -516684, -603084, -689484, -775884, -862284, 1716, 0 ] +}, { + "tz" : "Canada/Eastern", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2366736148 ], + "diffs" : [ 173852, 87452, 1052, -85348, -171748, -258148, -344548, -430948, -517348, -603748, -690148, -776548, -862948, 1052, 0 ] +}, { + "tz" : "Canada/Mountain", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 174832, 88432, 2032, -84368, -170768, -257168, -343568, -429968, -516368, -602768, -689168, -775568, -861968, 2032, 0 ] +}, { + "tz" : "Canada/Newfoundland", + "switches" : [ -62135757000, -59006435400, -55850675400, -52694915400, -46383395400, -43227635400, -40071875400, -33760355400, -30604595400, -27448835400, -21137315400, -17981555400, -14825795400, -12219280200, -2208988800 ], + "diffs" : [ 172852, 86452, 52, -86348, -172748, -259148, -345548, -431948, -518348, -604748, -691148, -777548, -863948, 52, 0 ] +}, { + "tz" : "Canada/Pacific", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2713880852 ], + "diffs" : [ 173548, 87148, 748, -85652, -172052, -258452, -344852, -431252, -517652, -604052, -690452, -776852, -863252, 748, 0 ] +}, { + "tz" : "Canada/Saskatchewan", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 176316, 89916, 3516, -82884, -169284, -255684, -342084, -428484, -514884, -601284, -687684, -774084, -860484, 3516, 0 ] +}, { + "tz" : "Canada/Yukon", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2208988800 ], + "diffs" : [ 176412, 90012, 3612, -82788, -169188, -255588, -341988, -428388, -514788, -601188, -687588, -773988, -860388, 3612, 0 ] +}, { + "tz" : "Chile/Continental", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400, -2208988800 ], + "diffs" : [ 175366, 88966, 2566, -83834, -170234, -256634, -343034, -429434, -515834, -602234, -688634, -775034, -861434, 2566, 0 ] +}, { + "tz" : "Chile/EasterIsland", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 177448, 91048, 4648, -81752, -168152, -254552, -340952, -427352, -513752, -600152, -686552, -772952, -859352, 4648, 0 ] +}, { + "tz" : "Cuba", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2524503608, -2208988800 ], + "diffs" : [ 174568, 88168, 1768, -84632, -171032, -257432, -343832, -430232, -516632, -603032, -689432, -775832, -862232, 1768, 1776, 0 ] +}, { + "tz" : "EET", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "EST5EDT", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Egypt", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172491, 86091, -309, -86709, -173109, -259509, -345909, -432309, -518709, -605109, -691509, -777909, -864309, -309, 0 ] +}, { + "tz" : "Eire", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2821651221, -2208988800 ], + "diffs" : [ 174300, 87900, 1500, -84900, -171300, -257700, -344100, -430500, -516900, -603300, -689700, -776100, -862500, 1500, 1521, 0 ] +}, { + "tz" : "Etc/GMT", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+0", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+1", + "switches" : [ -62135766000, -59006444400, -55850684400, -52694924400, -46383404400, -43227644400, -40071884400, -33760364400, -30604604400, -27448844400, -21137324400, -17981564400, -14825804400, -12219289200 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+10", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+11", + "switches" : [ -62135730000, -59006408400, -55850648400, -52694888400, -46383368400, -43227608400, -40071848400, -33760328400, -30604568400, -27448808400, -21137288400, -17981528400, -14825768400, -12219253200 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+12", + "switches" : [ -62135726400, -59006404800, -55850644800, -52694884800, -46383364800, -43227604800, -40071844800, -33760324800, -30604564800, -27448804800, -21137284800, -17981524800, -14825764800, -12219249600 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+2", + "switches" : [ -62135762400, -59006440800, -55850680800, -52694920800, -46383400800, -43227640800, -40071880800, -33760360800, -30604600800, -27448840800, -21137320800, -17981560800, -14825800800, -12219285600 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+3", + "switches" : [ -62135758800, -59006437200, -55850677200, -52694917200, -46383397200, -43227637200, -40071877200, -33760357200, -30604597200, -27448837200, -21137317200, -17981557200, -14825797200, -12219282000 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+4", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+5", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+6", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+7", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+8", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT+9", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-0", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-1", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-10", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-11", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-12", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-13", + "switches" : [ -62135816400, -59006494800, -55850734800, -52694974800, -46383454800, -43227694800, -40071934800, -33760414800, -30604654800, -27448894800, -21137374800, -17981614800, -14825854800, -12219339600 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-14", + "switches" : [ -62135820000, -59006498400, -55850738400, -52694978400, -46383458400, -43227698400, -40071938400, -33760418400, -30604658400, -27448898400, -21137378400, -17981618400, -14825858400, -12219343200 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-2", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-3", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-4", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-5", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-6", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-7", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-8", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT-9", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/GMT0", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/Greenwich", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/UCT", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/UTC", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/Universal", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Etc/Zulu", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Europe/Amsterdam", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175228, 88828, 2428, -83972, -170372, -256772, -343172, -429572, -515972, -602372, -688772, -775172, -861572, 2428, 0 ] +}, { + "tz" : "Europe/Andorra", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 176036, 89636, 3236, -83164, -169564, -255964, -342364, -428764, -515164, -601564, -687964, -774364, -860764, 3236, 0 ] +}, { + "tz" : "Europe/Astrakhan", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 175668, 89268, 2868, -83532, -169932, -256332, -342732, -429132, -515532, -601932, -688332, -774732, -861132, 2868, 0 ] +}, { + "tz" : "Europe/Athens", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 174308, 87908, 1508, -84892, -171292, -257692, -344092, -430492, -516892, -603292, -689692, -776092, -862492, 1508, 0 ] +}, { + "tz" : "Europe/Belfast", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -3852662325 ], + "diffs" : [ 172875, 86475, 75, -86325, -172725, -259125, -345525, -431925, -518325, -604725, -691125, -777525, -863925, 75, 0 ] +}, { + "tz" : "Europe/Belgrade", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2713915320 ], + "diffs" : [ 171480, 85080, -1320, -87720, -174120, -260520, -346920, -433320, -519720, -606120, -692520, -778920, -865320, -1320, 0 ] +}, { + "tz" : "Europe/Berlin", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2422054408 ], + "diffs" : [ 173192, 86792, 392, -86008, -172408, -258808, -345208, -431608, -518008, -604408, -690808, -777208, -863608, 392, 0 ] +}, { + "tz" : "Europe/Bratislava", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2469401864 ], + "diffs" : [ 172936, 86536, 136, -86264, -172664, -259064, -345464, -431864, -518264, -604664, -691064, -777464, -863864, 136, 0 ] +}, { + "tz" : "Europe/Brussels", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2450998800, -2208988800 ], + "diffs" : [ 175350, 88950, 2550, -83850, -170250, -256650, -343050, -429450, -515850, -602250, -688650, -775050, -861450, 2550, 3600, 0 ] +}, { + "tz" : "Europe/Bucharest", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 173736, 87336, 936, -85464, -171864, -258264, -344664, -431064, -517464, -603864, -690264, -776664, -863064, 936, 0 ] +}, { + "tz" : "Europe/Budapest", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2500938980 ], + "diffs" : [ 171820, 85420, -980, -87380, -173780, -260180, -346580, -432980, -519380, -605780, -692180, -778580, -864980, -980, 0 ] +}, { + "tz" : "Europe/Busingen", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -3675200662, -2385246586 ], + "diffs" : [ 174352, 87952, 1552, -84848, -171248, -257648, -344048, -430448, -516848, -603248, -689648, -776048, -862448, 1552, 1814, 0 ] +}, { + "tz" : "Europe/Chisinau", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2840148020, -2208988800 ], + "diffs" : [ 173080, 86680, 280, -86120, -172520, -258920, -345320, -431720, -518120, -604520, -690920, -777320, -863720, 280, 300, 0 ] +}, { + "tz" : "Europe/Copenhagen", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2398294220 ], + "diffs" : [ 173380, 86980, 580, -85820, -172220, -258620, -345020, -431420, -517820, -604220, -690620, -777020, -863420, 580, 0 ] +}, { + "tz" : "Europe/Dublin", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2821651221, -2208988800 ], + "diffs" : [ 174300, 87900, 1500, -84900, -171300, -257700, -344100, -430500, -516900, -603300, -689700, -776100, -862500, 1500, 1521, 0 ] +}, { + "tz" : "Europe/Gibraltar", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2821653516, -2208988800 ], + "diffs" : [ 177684, 91284, 4884, -81516, -167916, -254316, -340716, -427116, -513516, -599916, -686316, -772716, -859116, 4884, 3600, 0 ] +}, { + "tz" : "Europe/Guernsey", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -3852662325 ], + "diffs" : [ 172875, 86475, 75, -86325, -172725, -259125, -345525, -431925, -518325, -604725, -691125, -777525, -863925, 75, 0 ] +}, { + "tz" : "Europe/Helsinki", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 174011, 87611, 1211, -85189, -171589, -257989, -344389, -430789, -517189, -603589, -689989, -776389, -862789, 1211, 0 ] +}, { + "tz" : "Europe/Isle_of_Man", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -3852662325 ], + "diffs" : [ 172875, 86475, 75, -86325, -172725, -259125, -345525, -431925, -518325, -604725, -691125, -777525, -863925, 75, 0 ] +}, { + "tz" : "Europe/Istanbul", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2840151536, -2208988800 ], + "diffs" : [ 176648, 90248, 3848, -82552, -168952, -255352, -341752, -428152, -514552, -600952, -687352, -773752, -860152, 3848, 3784, 0 ] +}, { + "tz" : "Europe/Jersey", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -3852662325 ], + "diffs" : [ 172875, 86475, 75, -86325, -172725, -259125, -345525, -431925, -518325, -604725, -691125, -777525, -863925, 75, 0 ] +}, { + "tz" : "Europe/Kaliningrad", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2422059720, -2208988800 ], + "diffs" : [ 175080, 88680, 2280, -84120, -170520, -256920, -343320, -429720, -516120, -602520, -688920, -775320, -861720, 2280, 3600, 0 ] +}, { + "tz" : "Europe/Kiev", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 172676, 86276, -124, -86524, -172924, -259324, -345724, -432124, -518524, -604924, -691324, -777724, -864124, -124, 0 ] +}, { + "tz" : "Europe/Kirov", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 171672, 85272, -1128, -87528, -173928, -260328, -346728, -433128, -519528, -605928, -692328, -778728, -865128, -1128, 0 ] +}, { + "tz" : "Europe/Lisbon", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 175005, 88605, 2205, -84195, -170595, -256995, -343395, -429795, -516195, -602595, -688995, -775395, -861795, 2205, 0 ] +}, { + "tz" : "Europe/Ljubljana", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2713915320 ], + "diffs" : [ 171480, 85080, -1320, -87720, -174120, -260520, -346920, -433320, -519720, -606120, -692520, -778920, -865320, -1320, 0 ] +}, { + "tz" : "Europe/London", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -3852662325 ], + "diffs" : [ 172875, 86475, 75, -86325, -172725, -259125, -345525, -431925, -518325, -604725, -691125, -777525, -863925, 75, 0 ] +}, { + "tz" : "Europe/Luxembourg", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 174924, 88524, 2124, -84276, -170676, -257076, -343476, -429876, -516276, -602676, -689076, -775476, -861876, 2124, 0 ] +}, { + "tz" : "Europe/Madrid", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 177284, 90884, 4484, -81916, -168316, -254716, -341116, -427516, -513916, -600316, -686716, -773116, -859516, 4484, 0 ] +}, { + "tz" : "Europe/Malta", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2403478684 ], + "diffs" : [ 172916, 86516, 116, -86284, -172684, -259084, -345484, -431884, -518284, -604684, -691084, -777484, -863884, 116, 0 ] +}, { + "tz" : "Europe/Mariehamn", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 174011, 87611, 1211, -85189, -171589, -257989, -344389, -430789, -517189, -603589, -689989, -776389, -862789, 1211, 0 ] +}, { + "tz" : "Europe/Minsk", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2840151616, -2208988800 ], + "diffs" : [ 176984, 90584, 4184, -82216, -168616, -255016, -341416, -427816, -514216, -600616, -687016, -773416, -859816, 4184, 4200, 0 ] +}, { + "tz" : "Europe/Monaco", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2486683211, -2208988800 ], + "diffs" : [ 174628, 88228, 1828, -84572, -170972, -257372, -343772, -430172, -516572, -602972, -689372, -775772, -862172, 1828, 3039, 0 ] +}, { + "tz" : "Europe/Moscow", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174583, 88183, 1783, -84617, -171017, -257417, -343817, -430217, -516617, -603017, -689417, -775817, -862217, 1783, 0 ] +}, { + "tz" : "Europe/Nicosia", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 171992, 85592, -808, -87208, -173608, -260008, -346408, -432808, -519208, -605608, -692008, -778408, -864808, -808, 0 ] +}, { + "tz" : "Europe/Oslo", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2366757780 ], + "diffs" : [ 173820, 87420, 1020, -85380, -171780, -258180, -344580, -430980, -517380, -603780, -690180, -776580, -862980, 1020, 0 ] +}, { + "tz" : "Europe/Paris", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 175839, 89439, 3039, -83361, -169761, -256161, -342561, -428961, -515361, -601761, -688161, -774561, -860961, 3039, 0 ] +}, { + "tz" : "Europe/Podgorica", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2713915320 ], + "diffs" : [ 171480, 85080, -1320, -87720, -174120, -260520, -346920, -433320, -519720, -606120, -692520, -778920, -865320, -1320, 0 ] +}, { + "tz" : "Europe/Prague", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2469401864 ], + "diffs" : [ 172936, 86536, 136, -86264, -172664, -259064, -345464, -431864, -518264, -604664, -691064, -777464, -863864, 136, 0 ] +}, { + "tz" : "Europe/Riga", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 174206, 87806, 1406, -84994, -171394, -257794, -344194, -430594, -516994, -603394, -689794, -776194, -862594, 1406, 0 ] +}, { + "tz" : "Europe/Rome", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2403565200 ], + "diffs" : [ 173404, 87004, 604, -85796, -172196, -258596, -344996, -431396, -517796, -604196, -690596, -776996, -863396, 604, 0 ] +}, { + "tz" : "Europe/Samara", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 175180, 88780, 2380, -84020, -170420, -256820, -343220, -429620, -516020, -602420, -688820, -775220, -861620, 2380, 0 ] +}, { + "tz" : "Europe/San_Marino", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2403565200 ], + "diffs" : [ 173404, 87004, 604, -85796, -172196, -258596, -344996, -431396, -517796, -604196, -690596, -776996, -863396, 604, 0 ] +}, { + "tz" : "Europe/Sarajevo", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2713915320 ], + "diffs" : [ 171480, 85080, -1320, -87720, -174120, -260520, -346920, -433320, -519720, -606120, -692520, -778920, -865320, -1320, 0 ] +}, { + "tz" : "Europe/Saratov", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 176142, 89742, 3342, -83058, -169458, -255858, -342258, -428658, -515058, -601458, -687858, -774258, -860658, 3342, 0 ] +}, { + "tz" : "Europe/Simferopol", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2840151624, -2208988800 ], + "diffs" : [ 175416, 89016, 2616, -83784, -170184, -256584, -342984, -429384, -515784, -602184, -688584, -774984, -861384, 2616, 2640, 0 ] +}, { + "tz" : "Europe/Skopje", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2713915320 ], + "diffs" : [ 171480, 85080, -1320, -87720, -174120, -260520, -346920, -433320, -519720, -606120, -692520, -778920, -865320, -1320, 0 ] +}, { + "tz" : "Europe/Sofia", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2840146580, -2369527016 ], + "diffs" : [ 174404, 88004, 1604, -84796, -171196, -257596, -343996, -430396, -516796, -603196, -689596, -775996, -862396, 1604, 184, 0 ] +}, { + "tz" : "Europe/Stockholm", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2871681118, -2208992414 ], + "diffs" : [ 172068, 85668, -732, -87132, -173532, -259932, -346332, -432732, -519132, -605532, -691932, -778332, -864732, -732, -14, 0 ] +}, { + "tz" : "Europe/Tallinn", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 174060, 87660, 1260, -85140, -171540, -257940, -344340, -430740, -517140, -603540, -689940, -776340, -862740, 1260, 0 ] +}, { + "tz" : "Europe/Tirane", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 171640, 85240, -1160, -87560, -173960, -260360, -346760, -433160, -519560, -605960, -692360, -778760, -865160, -1160, 0 ] +}, { + "tz" : "Europe/Tiraspol", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2840148020, -2208988800 ], + "diffs" : [ 173080, 86680, 280, -86120, -172520, -258920, -345320, -431720, -518120, -604520, -690920, -777320, -863720, 280, 300, 0 ] +}, { + "tz" : "Europe/Ulyanovsk", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 175584, 89184, 2784, -83616, -170016, -256416, -342816, -429216, -515616, -602016, -688416, -774816, -861216, 2784, 0 ] +}, { + "tz" : "Europe/Uzhgorod", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2500943352, -2208988800 ], + "diffs" : [ 174648, 88248, 1848, -84552, -170952, -257352, -343752, -430152, -516552, -602952, -689352, -775752, -862152, 1848, 3600, 0 ] +}, { + "tz" : "Europe/Vaduz", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -3675200662, -2385246586 ], + "diffs" : [ 174352, 87952, 1552, -84848, -171248, -257648, -344048, -430448, -516848, -603248, -689648, -776048, -862448, 1552, 1814, 0 ] +}, { + "tz" : "Europe/Vatican", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2403565200 ], + "diffs" : [ 173404, 87004, 604, -85796, -172196, -258596, -344996, -431396, -517796, -604196, -690596, -776996, -863396, 604, 0 ] +}, { + "tz" : "Europe/Vienna", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2422055121 ], + "diffs" : [ 172479, 86079, -321, -86721, -173121, -259521, -345921, -432321, -518721, -605121, -691521, -777921, -864321, -321, 0 ] +}, { + "tz" : "Europe/Vilnius", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2840149036, -2208988800 ], + "diffs" : [ 173924, 87524, 1124, -85276, -171676, -258076, -344476, -430876, -517276, -603676, -690076, -776476, -862876, 1124, 2160, 0 ] +}, { + "tz" : "Europe/Volgograd", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 176540, 90140, 3740, -82660, -169060, -255460, -341860, -428260, -514660, -601060, -687460, -773860, -860260, 3740, 0 ] +}, { + "tz" : "Europe/Warsaw", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 171360, 84960, -1440, -87840, -174240, -260640, -347040, -433440, -519840, -606240, -692640, -779040, -865440, -1440, 0 ] +}, { + "tz" : "Europe/Zagreb", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2713915320 ], + "diffs" : [ 171480, 85080, -1320, -87720, -174120, -260520, -346920, -433320, -519720, -606120, -692520, -778920, -865320, -1320, 0 ] +}, { + "tz" : "Europe/Zaporozhye", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2840148040, -2208988800 ], + "diffs" : [ 171560, 85160, -1240, -87640, -174040, -260440, -346840, -433240, -519640, -606040, -692440, -778840, -865240, -1240, -1200, 0 ] +}, { + "tz" : "Europe/Zurich", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -3675200662, -2385246586 ], + "diffs" : [ 174352, 87952, 1552, -84848, -171248, -257648, -344048, -430448, -516848, -603248, -689648, -776048, -862448, 1552, 1814, 0 ] +}, { + "tz" : "GB", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -3852662325 ], + "diffs" : [ 172875, 86475, 75, -86325, -172725, -259125, -345525, -431925, -518325, -604725, -691125, -777525, -863925, 75, 0 ] +}, { + "tz" : "GB-Eire", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -3852662325 ], + "diffs" : [ 172875, 86475, 75, -86325, -172725, -259125, -345525, -431925, -518325, -604725, -691125, -777525, -863925, 75, 0 ] +}, { + "tz" : "GMT", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "GMT0", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Greenwich", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Hongkong", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 174198, 87798, 1398, -85002, -171402, -257802, -344202, -430602, -517002, -603402, -689802, -776202, -862602, 1398, 0 ] +}, { + "tz" : "Iceland", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 178080, 91680, 5280, -81120, -167520, -253920, -340320, -426720, -513120, -599520, -685920, -772320, -858720, 5280, 0 ] +}, { + "tz" : "Indian/Antananarivo", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Indian/Chagos", + "switches" : [ -62135791200, -59006469600, -55850709600, -52694949600, -46383429600, -43227669600, -40071909600, -33760389600, -30604629600, -27448869600, -21137349600, -17981589600, -14825829600, -12219314400, -2208988800 ], + "diffs" : [ 177020, 90620, 4220, -82180, -168580, -254980, -341380, -427780, -514180, -600580, -686980, -773380, -859780, 4220, 0 ] +}, { + "tz" : "Indian/Christmas", + "switches" : [ -62135794800, -59006473200, -55850713200, -52694953200, -46383433200, -43227673200, -40071913200, -33760393200, -30604633200, -27448873200, -21137353200, -17981593200, -14825833200, -12219318000, -2364102172 ], + "diffs" : [ 172628, 86228, -172, -86572, -172972, -259372, -345772, -432172, -518572, -604972, -691372, -777772, -864172, -172, 0 ] +}, { + "tz" : "Indian/Cocos", + "switches" : [ -62135793000, -59006471400, -55850711400, -52694951400, -46383431400, -43227671400, -40071911400, -33760391400, -30604631400, -27448871400, -21137351400, -17981591400, -14825831400, -12219316200, -2209012060 ], + "diffs" : [ 172940, 86540, 140, -86260, -172660, -259060, -345460, -431860, -518260, -604660, -691060, -777460, -863860, 140, 0 ] +}, { + "tz" : "Indian/Comoro", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Indian/Kerguelen", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 190800, 104400, 18000, -68400, -154800, -241200, -327600, -414000, -500400, -586800, -673200, -759600, -846000, 18000, 0 ] +}, { + "tz" : "Indian/Mahe", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 173892, 87492, 1092, -85308, -171708, -258108, -344508, -430908, -517308, -603708, -690108, -776508, -862908, 1092, 0 ] +}, { + "tz" : "Indian/Maldives", + "switches" : [ -62135787600, -59006466000, -55850706000, -52694946000, -46383426000, -43227666000, -40071906000, -33760386000, -30604626000, -27448866000, -21137346000, -17981586000, -14825826000, -12219310800, -2208988800 ], + "diffs" : [ 173160, 86760, 360, -86040, -172440, -258840, -345240, -431640, -518040, -604440, -690840, -777240, -863640, 360, 0 ] +}, { + "tz" : "Indian/Mauritius", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 173400, 87000, 600, -85800, -172200, -258600, -345000, -431400, -517800, -604200, -690600, -777000, -863400, 600, 0 ] +}, { + "tz" : "Indian/Mayotte", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174764, 88364, 1964, -84436, -170836, -257236, -343636, -430036, -516436, -602836, -689236, -775636, -862036, 1964, 0 ] +}, { + "tz" : "Indian/Reunion", + "switches" : [ -62135784000, -59006462400, -55850702400, -52694942400, -46383422400, -43227662400, -40071902400, -33760382400, -30604622400, -27448862400, -21137342400, -17981582400, -14825822400, -12219307200, -2208988800 ], + "diffs" : [ 173888, 87488, 1088, -85312, -171712, -258112, -344512, -430912, -517312, -603712, -690112, -776512, -862912, 1088, 0 ] +}, { + "tz" : "Israel", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2840148014, -2208988800 ], + "diffs" : [ 171546, 85146, -1254, -87654, -174054, -260454, -346854, -433254, -519654, -606054, -692454, -778854, -865254, -1254, -1240, 0 ] +}, { + "tz" : "Jamaica", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 173230, 86830, 430, -85970, -172370, -258770, -345170, -431570, -517970, -604370, -690770, -777170, -863570, 430, 0 ] +}, { + "tz" : "Japan", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2587712400 ], + "diffs" : [ 171661, 85261, -1139, -87539, -173939, -260339, -346739, -433139, -519539, -605939, -692339, -778739, -865139, -1139, 0 ] +}, { + "tz" : "Kwajalein", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 175840, 89440, 3040, -83360, -169760, -256160, -342560, -428960, -515360, -601760, -688160, -774560, -860960, 3040, 0 ] +}, { + "tz" : "Libya", + "switches" : [ -62135776800, -59006455200, -55850695200, -52694935200, -46383415200, -43227655200, -40071895200, -33760375200, -30604615200, -27448855200, -21137335200, -17981575200, -14825815200, -12219300000, -2208988800 ], + "diffs" : [ 176836, 90436, 4036, -82364, -168764, -255164, -341564, -427964, -514364, -600764, -687164, -773564, -859964, 4036, 0 ] +}, { + "tz" : "MET", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "MST7MDT", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Mexico/BajaNorte", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2208988800 ], + "diffs" : [ 172084, 85684, -716, -87116, -173516, -259916, -346316, -432716, -519116, -605516, -691916, -778316, -864716, -716, 0 ] +}, { + "tz" : "Mexico/BajaSur", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2208988800 ], + "diffs" : [ 173140, 86740, 340, -86060, -172460, -258860, -345260, -431660, -518060, -604460, -690860, -777260, -863660, 340, 0 ] +}, { + "tz" : "Mexico/General", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 174996, 88596, 2196, -84204, -170604, -257004, -343404, -429804, -516204, -602604, -689004, -775404, -861804, 2196, 0 ] +}, { + "tz" : "NZ", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -3192437344, -2208988800 ], + "diffs" : [ 174056, 87656, 1256, -85144, -171544, -257944, -344344, -430744, -517144, -603544, -689944, -776344, -862744, 1256, 1800, 0 ] +}, { + "tz" : "NZ-CHAT", + "switches" : [ -62135815500, -59006493900, -55850733900, -52694973900, -46383453900, -43227693900, -40071933900, -33760413900, -30604653900, -27448893900, -21137373900, -17981613900, -14825853900, -12219338700, -3192439428, -2208988800 ], + "diffs" : [ 174672, 88272, 1872, -84528, -170928, -257328, -343728, -430128, -516528, -602928, -689328, -775728, -862128, 1872, 1800, 0 ] +}, { + "tz" : "Navajo", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2717643600 ], + "diffs" : [ 172796, 86396, -4, -86404, -172804, -259204, -345604, -432004, -518404, -604804, -691204, -777604, -864004, -4, 0 ] +}, { + "tz" : "PRC", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 172457, 86057, -343, -86743, -173143, -259543, -345943, -432343, -518743, -605143, -691543, -777943, -864343, -343, 0 ] +}, { + "tz" : "PST8PDT", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Pacific/Apia", + "switches" : [ -62135816400, -59006494800, -55850734800, -52694974800, -46383454800, -43227694800, -40071934800, -33760414800, -30604654800, -27448894800, -21137374800, -17981614800, -14825854800, -12219339600, -2445512400, -2208988800 ], + "diffs" : [ 174416, 88016, 1616, -84784, -171184, -257584, -343984, -430384, -516784, -603184, -689584, -775984, -862384, 1616, 88016, 0 ] +}, { + "tz" : "Pacific/Auckland", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -3192437344, -2208988800 ], + "diffs" : [ 174056, 87656, 1256, -85144, -171544, -257944, -344344, -430744, -517144, -603544, -689944, -776344, -862744, 1256, 1800, 0 ] +}, { + "tz" : "Pacific/Bougainville", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -2840182424, -2366794112, -2208988800 ], + "diffs" : [ 175064, 88664, 2264, -84136, -170536, -256936, -343336, -429736, -516136, -602536, -688936, -775336, -861736, 2264, 4288, 3600, 0 ] +}, { + "tz" : "Pacific/Chatham", + "switches" : [ -62135815500, -59006493900, -55850733900, -52694973900, -46383453900, -43227693900, -40071933900, -33760413900, -30604653900, -27448893900, -21137373900, -17981613900, -14825853900, -12219338700, -3192439428, -2208988800 ], + "diffs" : [ 174672, 88272, 1872, -84528, -170928, -257328, -343728, -430128, -516528, -602928, -689328, -775728, -862128, 1872, 1800, 0 ] +}, { + "tz" : "Pacific/Chuuk", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -3944628000, -2208988800 ], + "diffs" : [ 258772, 172372, 85972, -428, -86828, -173228, -259628, -346028, -432428, -518828, -605228, -691628, -778028, 85972, -428, 0 ] +}, { + "tz" : "Pacific/Easter", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 177448, 91048, 4648, -81752, -168152, -254552, -340952, -427352, -513752, -600152, -686552, -772952, -859352, 4648, 0 ] +}, { + "tz" : "Pacific/Efate", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -2208988800 ], + "diffs" : [ 172004, 85604, -796, -87196, -173596, -259996, -346396, -432796, -519196, -605596, -691996, -778396, -864796, -796, 0 ] +}, { + "tz" : "Pacific/Enderbury", + "switches" : [ -62135816400, -59006494800, -55850734800, -52694974800, -46383454800, -43227694800, -40071934800, -33760414800, -30604654800, -27448894800, -21137374800, -17981614800, -14825854800, -12219339600, -2208988800 ], + "diffs" : [ 260660, 174260, 87860, 1460, -84940, -171340, -257740, -344140, -430540, -516940, -603340, -689740, -776140, 87860, 0 ] +}, { + "tz" : "Pacific/Fakaofo", + "switches" : [ -62135816400, -59006494800, -55850734800, -52694974800, -46383454800, -43227694800, -40071934800, -33760414800, -30604654800, -27448894800, -21137374800, -17981614800, -14825854800, -12219339600, -2208988800 ], + "diffs" : [ 260696, 174296, 87896, 1496, -84904, -171304, -257704, -344104, -430504, -516904, -603304, -689704, -776104, 87896, 0 ] +}, { + "tz" : "Pacific/Fiji", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 173056, 86656, 256, -86144, -172544, -258944, -345344, -431744, -518144, -604544, -690944, -777344, -863744, 256, 0 ] +}, { + "tz" : "Pacific/Funafuti", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 172988, 86588, 188, -86212, -172612, -259012, -345412, -431812, -518212, -604612, -691012, -777412, -863812, 188, 0 ] +}, { + "tz" : "Pacific/Galapagos", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2208988800 ], + "diffs" : [ 172704, 86304, -96, -86496, -172896, -259296, -345696, -432096, -518496, -604896, -691296, -777696, -864096, -96, 0 ] +}, { + "tz" : "Pacific/Gambier", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400, -2208988800 ], + "diffs" : [ 172788, 86388, -12, -86412, -172812, -259212, -345612, -432012, -518412, -604812, -691212, -777612, -864012, -12, 0 ] +}, { + "tz" : "Pacific/Guadalcanal", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -2208988800 ], + "diffs" : [ 174012, 87612, 1212, -85188, -171588, -257988, -344388, -430788, -517188, -603588, -689988, -776388, -862788, 1212, 0 ] +}, { + "tz" : "Pacific/Guam", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -3944628000, -2208988800 ], + "diffs" : [ 260460, 174060, 87660, 1260, -85140, -171540, -257940, -344340, -430740, -517140, -603540, -689940, -776340, 87660, 1260, 0 ] +}, { + "tz" : "Pacific/Honolulu", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800, -2334103114, -2208988800 ], + "diffs" : [ 174686, 88286, 1886, -84514, -170914, -257314, -343714, -430114, -516514, -602914, -689314, -775714, -862114, 1886, 1800, 0 ] +}, { + "tz" : "Pacific/Johnston", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800, -2334103114, -2208988800 ], + "diffs" : [ 174686, 88286, 1886, -84514, -170914, -257314, -343714, -430114, -516514, -602914, -689314, -775714, -862114, 1886, 1800, 0 ] +}, { + "tz" : "Pacific/Kiritimati", + "switches" : [ -62135820000, -59006498400, -55850738400, -52694978400, -46383458400, -43227698400, -40071938400, -33760418400, -30604658400, -27448898400, -21137378400, -17981618400, -14825858400, -12219343200, -2208988800 ], + "diffs" : [ 260960, 174560, 88160, 1760, -84640, -171040, -257440, -343840, -430240, -516640, -603040, -689440, -775840, 88160, 0 ] +}, { + "tz" : "Pacific/Kosrae", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -3944631600, -2208988800 ], + "diffs" : [ 259684, 173284, 86884, 484, -85916, -172316, -258716, -345116, -431516, -517916, -604316, -690716, -777116, 86884, 484, 0 ] +}, { + "tz" : "Pacific/Kwajalein", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 175840, 89440, 3040, -83360, -169760, -256160, -342560, -428960, -515360, -601760, -688160, -774560, -860960, 3040, 0 ] +}, { + "tz" : "Pacific/Majuro", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 174912, 88512, 2112, -84288, -170688, -257088, -343488, -429888, -516288, -602688, -689088, -775488, -861888, 2112, 0 ] +}, { + "tz" : "Pacific/Marquesas", + "switches" : [ -62135735400, -59006413800, -55850653800, -52694893800, -46383373800, -43227613800, -40071853800, -33760333800, -30604573800, -27448813800, -21137293800, -17981533800, -14825773800, -12219258600, -2208988800 ], + "diffs" : [ 172080, 85680, -720, -87120, -173520, -259920, -346320, -432720, -519120, -605520, -691920, -778320, -864720, -720, 0 ] +}, { + "tz" : "Pacific/Midway", + "switches" : [ -62135730000, -59006408400, -55850648400, -52694888400, -46383368400, -43227608400, -40071848400, -33760328400, -30604568400, -27448808400, -21137288400, -17981528400, -14825768400, -12219253200, -2445426000, -2208988800 ], + "diffs" : [ 87768, 1368, -85032, -171432, -257832, -344232, -430632, -517032, -603432, -689832, -776232, -862632, -949032, -85032, 1368, 0 ] +}, { + "tz" : "Pacific/Nauru", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 175940, 89540, 3140, -83260, -169660, -256060, -342460, -428860, -515260, -601660, -688060, -774460, -860860, 3140, 0 ] +}, { + "tz" : "Pacific/Niue", + "switches" : [ -62135730000, -59006408400, -55850648400, -52694888400, -46383368400, -43227608400, -40071848400, -33760328400, -30604568400, -27448808400, -21137288400, -17981528400, -14825768400, -12219253200, -2208988800 ], + "diffs" : [ 173980, 87580, 1180, -85220, -171620, -258020, -344420, -430820, -517220, -603620, -690020, -776420, -862820, 1180, 0 ] +}, { + "tz" : "Pacific/Norfolk", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -2208988800 ], + "diffs" : [ 172088, 85688, -712, -87112, -173512, -259912, -346312, -432712, -519112, -605512, -691912, -778312, -864712, -712, 0 ] +}, { + "tz" : "Pacific/Noumea", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -2208988800 ], + "diffs" : [ 172452, 86052, -348, -86748, -173148, -259548, -345948, -432348, -518748, -605148, -691548, -777948, -864348, -348, 0 ] +}, { + "tz" : "Pacific/Pago_Pago", + "switches" : [ -62135730000, -59006408400, -55850648400, -52694888400, -46383368400, -43227608400, -40071848400, -33760328400, -30604568400, -27448808400, -21137288400, -17981528400, -14825768400, -12219253200, -2445426000, -2208988800 ], + "diffs" : [ 87768, 1368, -85032, -171432, -257832, -344232, -430632, -517032, -603432, -689832, -776232, -862632, -949032, -85032, 1368, 0 ] +}, { + "tz" : "Pacific/Palau", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -3944624400, -2208988800 ], + "diffs" : [ 259324, 172924, 86524, 124, -86276, -172676, -259076, -345476, -431876, -518276, -604676, -691076, -777476, 86524, 124, 0 ] +}, { + "tz" : "Pacific/Pitcairn", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2208988800 ], + "diffs" : [ 175220, 88820, 2420, -83980, -170380, -256780, -343180, -429580, -515980, -602380, -688780, -775180, -861580, 2420, 0 ] +}, { + "tz" : "Pacific/Pohnpei", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -3944631600, -2208988800 ], + "diffs" : [ 260828, 174428, 88028, 1628, -84772, -171172, -257572, -343972, -430372, -516772, -603172, -689572, -775972, 88028, 1628, 0 ] +}, { + "tz" : "Pacific/Ponape", + "switches" : [ -62135809200, -59006487600, -55850727600, -52694967600, -46383447600, -43227687600, -40071927600, -33760407600, -30604647600, -27448887600, -21137367600, -17981607600, -14825847600, -12219332400, -3944631600, -2208988800 ], + "diffs" : [ 260828, 174428, 88028, 1628, -84772, -171172, -257572, -343972, -430372, -516772, -603172, -689572, -775972, 88028, 1628, 0 ] +}, { + "tz" : "Pacific/Port_Moresby", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -2840176808, -2366790512 ], + "diffs" : [ 173480, 87080, 680, -85720, -172120, -258520, -344920, -431320, -517720, -604120, -690520, -776920, -863320, 680, 688, 0 ] +}, { + "tz" : "Pacific/Rarotonga", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800, -2208988800 ], + "diffs" : [ 175144, 88744, 2344, -84056, -170456, -256856, -343256, -429656, -516056, -602456, -688856, -775256, -861656, 2344, 0 ] +}, { + "tz" : "Pacific/Saipan", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -3944628000, -2208988800 ], + "diffs" : [ 260460, 174060, 87660, 1260, -85140, -171540, -257940, -344340, -430740, -517140, -603540, -689940, -776340, 87660, 1260, 0 ] +}, { + "tz" : "Pacific/Samoa", + "switches" : [ -62135730000, -59006408400, -55850648400, -52694888400, -46383368400, -43227608400, -40071848400, -33760328400, -30604568400, -27448808400, -21137288400, -17981528400, -14825768400, -12219253200, -2445426000, -2208988800 ], + "diffs" : [ 87768, 1368, -85032, -171432, -257832, -344232, -430632, -517032, -603432, -689832, -776232, -862632, -949032, -85032, 1368, 0 ] +}, { + "tz" : "Pacific/Tahiti", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800, -2208988800 ], + "diffs" : [ 172696, 86296, -104, -86504, -172904, -259304, -345704, -432104, -518504, -604904, -691304, -777704, -864104, -104, 0 ] +}, { + "tz" : "Pacific/Tarawa", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 174476, 88076, 1676, -84724, -171124, -257524, -343924, -430324, -516724, -603124, -689524, -775924, -862324, 1676, 0 ] +}, { + "tz" : "Pacific/Tongatapu", + "switches" : [ -62135816400, -59006494800, -55850734800, -52694974800, -46383454800, -43227694800, -40071934800, -33760414800, -30604654800, -27448894800, -21137374800, -17981614800, -14825854800, -12219339600, -2208988800 ], + "diffs" : [ 175240, 88840, 2440, -83960, -170360, -256760, -343160, -429560, -515960, -602360, -688760, -775160, -861560, 2440, 0 ] +}, { + "tz" : "Pacific/Truk", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -3944628000, -2208988800 ], + "diffs" : [ 258772, 172372, 85972, -428, -86828, -173228, -259628, -346028, -432428, -518828, -605228, -691628, -778028, 85972, -428, 0 ] +}, { + "tz" : "Pacific/Wake", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 176012, 89612, 3212, -83188, -169588, -255988, -342388, -428788, -515188, -601588, -687988, -774388, -860788, 3212, 0 ] +}, { + "tz" : "Pacific/Wallis", + "switches" : [ -62135812800, -59006491200, -55850731200, -52694971200, -46383451200, -43227691200, -40071931200, -33760411200, -30604651200, -27448891200, -21137371200, -17981611200, -14825851200, -12219336000, -2208988800 ], + "diffs" : [ 171880, 85480, -920, -87320, -173720, -260120, -346520, -432920, -519320, -605720, -692120, -778520, -864920, -920, 0 ] +}, { + "tz" : "Pacific/Yap", + "switches" : [ -62135805600, -59006484000, -55850724000, -52694964000, -46383444000, -43227684000, -40071924000, -33760404000, -30604644000, -27448884000, -21137364000, -17981604000, -14825844000, -12219328800, -3944628000, -2208988800 ], + "diffs" : [ 258772, 172372, 85972, -428, -86828, -173228, -259628, -346028, -432428, -518828, -605228, -691628, -778028, 85972, -428, 0 ] +}, { + "tz" : "Poland", + "switches" : [ -62135773200, -59006451600, -55850691600, -52694931600, -46383411600, -43227651600, -40071891600, -33760371600, -30604611600, -27448851600, -21137331600, -17981571600, -14825811600, -12219296400, -2208988800 ], + "diffs" : [ 171360, 84960, -1440, -87840, -174240, -260640, -347040, -433440, -519840, -606240, -692640, -779040, -865440, -1440, 0 ] +}, { + "tz" : "Portugal", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800, -2208988800 ], + "diffs" : [ 175005, 88605, 2205, -84195, -170595, -256995, -343395, -429795, -516195, -602595, -688995, -775395, -861795, 2205, 0 ] +}, { + "tz" : "ROK", + "switches" : [ -62135802000, -59006480400, -55850720400, -52694960400, -46383440400, -43227680400, -40071920400, -33760400400, -30604640400, -27448880400, -21137360400, -17981600400, -14825840400, -12219325200, -2208988800 ], + "diffs" : [ 174728, 88328, 1928, -84472, -170872, -257272, -343672, -430072, -516472, -602872, -689272, -775672, -862072, 1928, 0 ] +}, { + "tz" : "Singapore", + "switches" : [ -62135798400, -59006476800, -55850716800, -52694956800, -46383436800, -43227676800, -40071916800, -33760396800, -30604636800, -27448876800, -21137356800, -17981596800, -14825836800, -12219321600, -2208988800 ], + "diffs" : [ 176675, 90275, 3875, -82525, -168925, -255325, -341725, -428125, -514525, -600925, -687325, -773725, -860125, 3875, 0 ] +}, { + "tz" : "SystemV/AST4", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/AST4ADT", + "switches" : [ -62135755200, -59006433600, -55850673600, -52694913600, -46383393600, -43227633600, -40071873600, -33760353600, -30604593600, -27448833600, -21137313600, -17981553600, -14825793600, -12219278400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/CST6", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/CST6CDT", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/EST5", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/EST5EDT", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/HST10", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/MST7", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/MST7MDT", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/PST8", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/PST8PDT", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/YST9", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "SystemV/YST9YDT", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Turkey", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2840151536, -2208988800 ], + "diffs" : [ 176648, 90248, 3848, -82552, -168952, -255352, -341752, -428152, -514552, -600952, -687352, -773752, -860152, 3848, 3784, 0 ] +}, { + "tz" : "UCT", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "US/Alaska", + "switches" : [ -62135737200, -59006415600, -55850655600, -52694895600, -46383375600, -43227615600, -40071855600, -33760335600, -30604575600, -27448815600, -21137295600, -17981535600, -14825775600, -12219260400, -3225227303, -2208988800 ], + "diffs" : [ 89976, 3576, -82824, -169224, -255624, -342024, -428424, -514824, -601224, -687624, -774024, -860424, -946824, -82824, 3576, 0 ] +}, { + "tz" : "US/Aleutian", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800, -3225230125, -2208988800 ], + "diffs" : [ 92798, 6398, -80002, -166402, -252802, -339202, -425602, -512002, -598402, -684802, -771202, -857602, -944002, -80002, 6398, 0 ] +}, { + "tz" : "US/Arizona", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2717643600 ], + "diffs" : [ 174498, 88098, 1698, -84702, -171102, -257502, -343902, -430302, -516702, -603102, -689502, -775902, -862302, 1698, 0 ] +}, { + "tz" : "US/Central", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2717647200 ], + "diffs" : [ 172236, 85836, -564, -86964, -173364, -259764, -346164, -432564, -518964, -605364, -691764, -778164, -864564, -564, 0 ] +}, { + "tz" : "US/East-Indiana", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800, -2208988800 ], + "diffs" : [ 175478, 89078, 2678, -83722, -170122, -256522, -342922, -429322, -515722, -602122, -688522, -774922, -861322, 2678, 3600, 0 ] +}, { + "tz" : "US/Eastern", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2717650800 ], + "diffs" : [ 172562, 86162, -238, -86638, -173038, -259438, -345838, -432238, -518638, -605038, -691438, -777838, -864238, -238, 0 ] +}, { + "tz" : "US/Hawaii", + "switches" : [ -62135733600, -59006412000, -55850652000, -52694892000, -46383372000, -43227612000, -40071852000, -33760332000, -30604572000, -27448812000, -21137292000, -17981532000, -14825772000, -12219256800, -2334103114, -2208988800 ], + "diffs" : [ 174686, 88286, 1886, -84514, -170914, -257314, -343714, -430114, -516514, -602914, -689314, -775714, -862114, 1886, 1800, 0 ] +}, { + "tz" : "US/Indiana-Starke", + "switches" : [ -62135748000, -59006426400, -55850666400, -52694906400, -46383386400, -43227626400, -40071866400, -33760346400, -30604586400, -27448826400, -21137306400, -17981546400, -14825786400, -12219271200, -2717647200 ], + "diffs" : [ 171990, 85590, -810, -87210, -173610, -260010, -346410, -432810, -519210, -605610, -692010, -778410, -864810, -810, 0 ] +}, { + "tz" : "US/Michigan", + "switches" : [ -62135751600, -59006430000, -55850670000, -52694910000, -46383390000, -43227630000, -40071870000, -33760350000, -30604590000, -27448830000, -21137310000, -17981550000, -14825790000, -12219274800, -2208988800 ], + "diffs" : [ 174731, 88331, 1931, -84469, -170869, -257269, -343669, -430069, -516469, -602869, -689269, -775669, -862069, 1931, 0 ] +}, { + "tz" : "US/Mountain", + "switches" : [ -62135744400, -59006422800, -55850662800, -52694902800, -46383382800, -43227622800, -40071862800, -33760342800, -30604582800, -27448822800, -21137302800, -17981542800, -14825782800, -12219267600, -2717643600 ], + "diffs" : [ 172796, 86396, -4, -86404, -172804, -259204, -345604, -432004, -518404, -604804, -691204, -777604, -864004, -4, 0 ] +}, { + "tz" : "US/Pacific", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2717640000 ], + "diffs" : [ 172378, 85978, -422, -86822, -173222, -259622, -346022, -432422, -518822, -605222, -691622, -778022, -864422, -422, 0 ] +}, { + "tz" : "US/Pacific-New", + "switches" : [ -62135740800, -59006419200, -55850659200, -52694899200, -46383379200, -43227619200, -40071859200, -33760339200, -30604579200, -27448819200, -21137299200, -17981539200, -14825779200, -12219264000, -2717640000 ], + "diffs" : [ 172378, 85978, -422, -86822, -173222, -259622, -346022, -432422, -518822, -605222, -691622, -778022, -864422, -422, 0 ] +}, { + "tz" : "US/Samoa", + "switches" : [ -62135730000, -59006408400, -55850648400, -52694888400, -46383368400, -43227608400, -40071848400, -33760328400, -30604568400, -27448808400, -21137288400, -17981528400, -14825768400, -12219253200, -2445426000, -2208988800 ], + "diffs" : [ 87768, 1368, -85032, -171432, -257832, -344232, -430632, -517032, -603432, -689832, -776232, -862632, -949032, -85032, 1368, 0 ] +}, { + "tz" : "UTC", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Universal", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "W-SU", + "switches" : [ -62135780400, -59006458800, -55850698800, -52694938800, -46383418800, -43227658800, -40071898800, -33760378800, -30604618800, -27448858800, -21137338800, -17981578800, -14825818800, -12219303600, -2208988800 ], + "diffs" : [ 174583, 88183, 1783, -84617, -171017, -257417, -343817, -430217, -516617, -603017, -689417, -775817, -862217, 1783, 0 ] +}, { + "tz" : "WET", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +}, { + "tz" : "Zulu", + "switches" : [ -62135769600, -59006448000, -55850688000, -52694928000, -46383408000, -43227648000, -40071888000, -33760368000, -30604608000, -27448848000, -21137328000, -17981568000, -14825808000, -12219292800 ], + "diffs" : [ 172800, 86400, 0, -86400, -172800, -259200, -345600, -432000, -518400, -604800, -691200, -777600, -864000, 0 ] +} ] \ No newline at end of file diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 1f7634bafa420..9c8da32cbc0ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst +import javax.lang.model.SourceVersion + import org.apache.commons.lang3.reflect.ConstructorUtils import org.apache.spark.internal.Logging @@ -539,9 +541,10 @@ object ScalaReflection extends ScalaReflection { val params = getConstructorParameters(t) val fields = params.map { case (fieldName, fieldType) => - if (javaKeywords.contains(fieldName)) { - throw new UnsupportedOperationException(s"`$fieldName` is a reserved keyword and " + - "cannot be used as field name\n" + walkedTypePath) + if (SourceVersion.isKeyword(fieldName) || + !SourceVersion.isIdentifier(encodeFieldNameToIdentifier(fieldName))) { + throw new UnsupportedOperationException(s"`$fieldName` is not a valid identifier of " + + "Java and cannot be used as field name\n" + walkedTypePath) } // SPARK-26730 inputObject won't be null with If's guard below. And KnownNotNul @@ -611,10 +614,39 @@ object ScalaReflection extends ScalaReflection { } } + private def erasure(tpe: Type): Type = { + // For user-defined AnyVal classes, we should not erasure it. Otherwise, it will + // resolve to underlying type which wrapped by this class, e.g erasure + // `case class Foo(i: Int) extends AnyVal` will return type `Int` instead of `Foo`. + // But, for other types, we do need to erasure it. For example, we need to erasure + // `scala.Any` to `java.lang.Object` in order to load it from Java ClassLoader. + // Please see SPARK-17368 & SPARK-31190 for more details. + if (isSubtype(tpe, localTypeOf[AnyVal]) && !tpe.toString.startsWith("scala")) { + tpe + } else { + tpe.erasure + } + } + + /** + * Returns the full class name for a type. The returned name is the canonical + * Scala name, where each component is separated by a period. It is NOT the + * Java-equivalent runtime name (no dollar signs). + * + * In simple cases, both the Scala and Java names are the same, however when Scala + * generates constructs that do not map to a Java equivalent, such as singleton objects + * or nested classes in package objects, it uses the dollar sign ($) to create + * synthetic classes, emulating behaviour in Java bytecode. + */ + def getClassNameFromType(tpe: `Type`): String = { + erasure(tpe).dealias.typeSymbol.asClass.fullName + } + /* * Retrieves the runtime class corresponding to the provided type. */ - def getClassFromType(tpe: Type): Class[_] = mirror.runtimeClass(tpe.dealias.typeSymbol.asClass) + def getClassFromType(tpe: Type): Class[_] = + mirror.runtimeClass(erasure(tpe).dealias.typeSymbol.asClass) case class Schema(dataType: DataType, nullable: Boolean) @@ -671,6 +703,8 @@ object ScalaReflection extends ScalaReflection { Schema(TimestampType, nullable = true) case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => Schema(DateType, nullable = true) case t if isSubtype(t, localTypeOf[java.sql.Date]) => Schema(DateType, nullable = true) + case t if isSubtype(t, localTypeOf[CalendarInterval]) => + Schema(CalendarIntervalType, nullable = true) case t if isSubtype(t, localTypeOf[BigDecimal]) => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => @@ -753,13 +787,6 @@ object ScalaReflection extends ScalaReflection { } } - private val javaKeywords = Set("abstract", "assert", "boolean", "break", "byte", "case", "catch", - "char", "class", "const", "continue", "default", "do", "double", "else", "extends", "false", - "final", "finally", "float", "for", "goto", "if", "implements", "import", "instanceof", "int", - "interface", "long", "native", "new", "null", "package", "private", "protected", "public", - "return", "short", "static", "strictfp", "super", "switch", "synchronized", "this", "throw", - "throws", "transient", "true", "try", "void", "volatile", "while") - val typeJavaMapping = Map[DataType, Class[_]]( BooleanType -> classOf[Boolean], ByteType -> classOf[Byte], @@ -818,6 +845,10 @@ object ScalaReflection extends ScalaReflection { Seq.empty } } + + def encodeFieldNameToIdentifier(fieldName: String): String = { + TermName(fieldName).encodedName.toString + } } /** @@ -863,20 +894,6 @@ trait ScalaReflection extends Logging { tag.in(mirror).tpe.dealias } - /** - * Returns the full class name for a type. The returned name is the canonical - * Scala name, where each component is separated by a period. It is NOT the - * Java-equivalent runtime name (no dollar signs). - * - * In simple cases, both the Scala and Java names are the same, however when Scala - * generates constructs that do not map to a Java equivalent, such as singleton objects - * or nested classes in package objects, it uses the dollar sign ($) to create - * synthetic classes, emulating behaviour in Java bytecode. - */ - def getClassNameFromType(tpe: `Type`): String = { - tpe.dealias.erasure.typeSymbol.asClass.fullName - } - /** * Returns the parameter names and types for the primary constructor of this type. * 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 e9a8fc1540fac..09a25d0f02e2a 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 @@ -296,8 +296,8 @@ class Analyzer( case (CalendarIntervalType, CalendarIntervalType) => a case (_, CalendarIntervalType) => Cast(TimeAdd(l, r), l.dataType) case (CalendarIntervalType, _) => Cast(TimeAdd(r, l), r.dataType) - case (DateType, _) => DateAdd(l, r) - case (_, DateType) => DateAdd(r, l) + case (DateType, dt) if dt != StringType => DateAdd(l, r) + case (dt, DateType) if dt != StringType => DateAdd(r, l) case _ => a } case s @ Subtract(l, r) if s.childrenResolved => (l.dataType, r.dataType) match { @@ -306,7 +306,7 @@ class Analyzer( case (TimestampType, _) => SubtractTimestamps(l, r) case (_, TimestampType) => SubtractTimestamps(l, r) case (_, DateType) => SubtractDates(l, r) - case (DateType, _) => DateSub(l, r) + case (DateType, dt) if dt != StringType => DateSub(l, r) case _ => s } case m @ Multiply(l, r) if m.childrenResolved => (l.dataType, r.dataType) match { @@ -748,6 +748,8 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case s @ ShowTables(UnresolvedNamespace(Seq()), _) => s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) + case s @ ShowViews(UnresolvedNamespace(Seq()), _) => + s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) case UnresolvedNamespace(Seq()) => ResolvedNamespace(currentCatalog, Seq.empty[String]) case UnresolvedNamespace(CatalogAndNamespace(catalog, ns)) => @@ -2725,13 +2727,13 @@ class Analyzer( case p => p transformExpressionsUp { - case udf @ ScalaUDF(_, _, inputs, inputPrimitives, _, _, _, _) - if inputPrimitives.contains(true) => + case udf @ ScalaUDF(_, _, inputs, _, _, _, _) + if udf.inputPrimitives.contains(true) => // Otherwise, add special handling of null for fields that can't accept null. // The result of operations like this, when passed null, is generally to return null. - assert(inputPrimitives.length == inputs.length) + assert(udf.inputPrimitives.length == inputs.length) - val inputPrimitivesPair = inputPrimitives.zip(inputs) + val inputPrimitivesPair = udf.inputPrimitives.zip(inputs) val inputNullCheck = inputPrimitivesPair.collect { case (isPrimitive, input) if isPrimitive && input.nullable => IsNull(input) 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 8cdb451c2f40e..0891bc82f3e3b 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 @@ -200,10 +200,6 @@ trait CheckAnalysis extends PredicateHelper { s"filter expression '${f.condition.sql}' " + s"of type ${f.condition.dataType.catalogString} is not a boolean.") - case Filter(condition, _) if hasNullAwarePredicateWithinNot(condition) => - failAnalysis("Null-aware predicate sub-queries cannot be used in nested " + - s"conditions: $condition") - case j @ Join(_, _, _, Some(condition), _) if condition.dataType != BooleanType => failAnalysis( s"join condition '${condition.sql}' " + 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 6c4aee4f58b75..aba755cec8990 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 @@ -218,7 +218,7 @@ object FunctionRegistry { expression[PosExplode]("posexplode"), expressionGeneratorOuter[PosExplode]("posexplode_outer"), expression[Rand]("rand"), - expression[Rand]("random"), + expression[Rand]("random", true), expression[Randn]("randn"), expression[Stack]("stack"), expression[CaseWhen]("when"), @@ -235,7 +235,7 @@ object FunctionRegistry { expression[BRound]("bround"), expression[Cbrt]("cbrt"), expression[Ceil]("ceil"), - expression[Ceil]("ceiling"), + expression[Ceil]("ceiling", true), expression[Cos]("cos"), expression[Cosh]("cosh"), expression[Conv]("conv"), @@ -252,12 +252,12 @@ object FunctionRegistry { expression[Log1p]("log1p"), expression[Log2]("log2"), expression[Log]("ln"), - expression[Remainder]("mod"), + expression[Remainder]("mod", true), expression[UnaryMinus]("negative"), expression[Pi]("pi"), expression[Pmod]("pmod"), expression[UnaryPositive]("positive"), - expression[Pow]("pow"), + expression[Pow]("pow", true), expression[Pow]("power"), expression[ToRadians]("radians"), expression[Rint]("rint"), @@ -265,7 +265,7 @@ object FunctionRegistry { expression[ShiftLeft]("shiftleft"), expression[ShiftRight]("shiftright"), expression[ShiftRightUnsigned]("shiftrightunsigned"), - expression[Signum]("sign"), + expression[Signum]("sign", true), expression[Signum]("signum"), expression[Sin]("sin"), expression[Sinh]("sinh"), @@ -323,12 +323,12 @@ object FunctionRegistry { // string functions expression[Ascii]("ascii"), - expression[Chr]("char"), + expression[Chr]("char", true), expression[Chr]("chr"), expression[Base64]("base64"), expression[BitLength]("bit_length"), - expression[Length]("char_length"), - expression[Length]("character_length"), + expression[Length]("char_length", true), + expression[Length]("character_length", true), expression[ConcatWs]("concat_ws"), expression[Decode]("decode"), expression[Elt]("elt"), @@ -351,7 +351,7 @@ object FunctionRegistry { expression[JsonTuple]("json_tuple"), expression[ParseUrl]("parse_url"), expression[StringLocate]("position"), - expression[FormatString]("printf"), + expression[FormatString]("printf", true), expression[RegExpExtract]("regexp_extract"), expression[RegExpReplace]("regexp_replace"), expression[StringRepeat]("repeat"), @@ -364,21 +364,21 @@ object FunctionRegistry { expression[SoundEx]("soundex"), expression[StringSpace]("space"), expression[StringSplit]("split"), - expression[Substring]("substr"), + expression[Substring]("substr", true), expression[Substring]("substring"), expression[Left]("left"), expression[Right]("right"), expression[SubstringIndex]("substring_index"), expression[StringTranslate]("translate"), expression[StringTrim]("trim"), - expression[Upper]("ucase"), + expression[Upper]("ucase", true), expression[UnBase64]("unbase64"), expression[Unhex]("unhex"), expression[Upper]("upper"), expression[XPathList]("xpath"), expression[XPathBoolean]("xpath_boolean"), expression[XPathDouble]("xpath_double"), - expression[XPathDouble]("xpath_number"), + expression[XPathDouble]("xpath_number", true), expression[XPathFloat]("xpath_float"), expression[XPathInt]("xpath_int"), expression[XPathLong]("xpath_long"), @@ -393,7 +393,7 @@ object FunctionRegistry { expression[DateAdd]("date_add"), expression[DateFormatClass]("date_format"), expression[DateSub]("date_sub"), - expression[DayOfMonth]("day"), + expression[DayOfMonth]("day", true), expression[DayOfYear]("dayofyear"), expression[DayOfMonth]("dayofmonth"), expression[FromUnixTime]("from_unixtime"), @@ -404,7 +404,7 @@ object FunctionRegistry { expression[Month]("month"), expression[MonthsBetween]("months_between"), expression[NextDay]("next_day"), - expression[CurrentTimestamp]("now"), + expression[CurrentTimestamp]("now", true), expression[Quarter]("quarter"), expression[Second]("second"), expression[ParseToTimestamp]("to_timestamp"), @@ -445,7 +445,7 @@ object FunctionRegistry { expression[MapConcat]("map_concat"), expression[Size]("size"), expression[Slice]("slice"), - expression[Size]("cardinality"), + expression[Size]("cardinality", true), expression[ArraysZip]("arrays_zip"), expression[SortArray]("sort_array"), expression[Shuffle]("shuffle"), @@ -478,7 +478,7 @@ object FunctionRegistry { expression[Uuid]("uuid"), expression[Murmur3Hash]("hash"), expression[XxHash64]("xxhash64"), - expression[Sha1]("sha"), + expression[Sha1]("sha", true), expression[Sha1]("sha1"), expression[Sha2]("sha2"), expression[SparkPartitionID]("spark_partition_id"), @@ -488,7 +488,7 @@ object FunctionRegistry { expression[MonotonicallyIncreasingID]("monotonically_increasing_id"), expression[CurrentDatabase]("current_database"), expression[CallMethodViaReflection]("reflect"), - expression[CallMethodViaReflection]("java_method"), + expression[CallMethodViaReflection]("java_method", true), expression[SparkVersion]("version"), expression[TypeOf]("typeof"), @@ -538,6 +538,8 @@ object FunctionRegistry { expression[StructsToJson]("to_json"), expression[JsonToStructs]("from_json"), expression[SchemaOfJson]("schema_of_json"), + expression[LengthOfJsonArray]("json_array_length"), + expression[JsonObjectKeys]("json_object_keys"), // cast expression[Cast]("cast"), @@ -590,7 +592,9 @@ object FunctionRegistry { if (varargCtor.isDefined) { // If there is an apply method that accepts Seq[Expression], use that one. try { - varargCtor.get.newInstance(expressions).asInstanceOf[Expression] + val exp = varargCtor.get.newInstance(expressions).asInstanceOf[Expression] + if (setAlias) exp.setTagValue(FUNC_ALIAS, name) + exp } catch { // the exception is an invocation exception. To get a meaningful message, we need the // cause. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala index c6e0c74527b8f..71c6d40dfba16 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala @@ -24,15 +24,16 @@ import org.apache.spark.sql.catalyst.plans.logical.{HintErrorHandler, HintInfo} * The hint error handler that logs warnings for each hint error. */ object HintErrorLogger extends HintErrorHandler with Logging { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override def hintNotRecognized(name: String, parameters: Seq[Any]): Unit = { logWarning(s"Unrecognized hint: ${hintToPrettyString(name, parameters)}") } override def hintRelationsNotFound( - name: String, parameters: Seq[Any], invalidRelations: Set[String]): Unit = { - invalidRelations.foreach { n => - logWarning(s"Count not find relation '$n' specified in hint " + + name: String, parameters: Seq[Any], invalidRelations: Set[Seq[String]]): Unit = { + invalidRelations.foreach { ident => + logWarning(s"Count not find relation '${ident.quoted}' specified in hint " + s"'${hintToPrettyString(name, parameters)}'.") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 96558410d4004..2a0a944e4849c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -34,6 +34,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case AlterTableAddColumnsStatement( nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => + cols.foreach(c => failCharType(c.dataType)) val changes = cols.map { col => TableChange.addColumn( col.name.toArray, @@ -46,6 +47,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case AlterTableReplaceColumnsStatement( nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => + cols.foreach(c => failCharType(c.dataType)) val changes: Seq[TableChange] = loadTable(catalog, tbl.asIdentifier) match { case Some(table) => // REPLACE COLUMNS deletes all the existing columns and adds new columns specified. @@ -67,6 +69,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case a @ AlterTableAlterColumnStatement( nameParts @ NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => + a.dataType.foreach(failCharType) val colName = a.column.toArray val typeChange = a.dataType.map { newDataType => TableChange.updateColumnType(colName, newDataType) @@ -142,6 +145,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case c @ CreateTableStatement( NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + assertNoCharTypeInSchema(c.tableSchema) CreateV2Table( catalog.asTableCatalog, tbl.asIdentifier, @@ -152,7 +156,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) ignoreIfExists = c.ifNotExists) case c @ CreateTableAsSelectStatement( - NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _, _) => CreateTableAsSelect( catalog.asTableCatalog, tbl.asIdentifier, @@ -160,7 +164,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), - writeOptions = c.options, + writeOptions = c.writeOptions, ignoreIfExists = c.ifNotExists) case RefreshTableStatement(NonSessionCatalogAndTable(catalog, tbl)) => @@ -168,6 +172,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case c @ ReplaceTableStatement( NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + assertNoCharTypeInSchema(c.tableSchema) ReplaceTable( catalog.asTableCatalog, tbl.asIdentifier, @@ -178,7 +183,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) orCreate = c.orCreate) case c @ ReplaceTableAsSelectStatement( - NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _, _) => ReplaceTableAsSelect( catalog.asTableCatalog, tbl.asIdentifier, @@ -186,7 +191,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), - writeOptions = c.options, + writeOptions = c.writeOptions, orCreate = c.orCreate) case DropTableStatement(NonSessionCatalogAndTable(catalog, tbl), ifExists, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index 5b77d67bd1340..81de086e78f91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -64,31 +64,59 @@ object ResolveHints { _.toUpperCase(Locale.ROOT)).contains(hintName.toUpperCase(Locale.ROOT)))) } + // This method checks if given multi-part identifiers are matched with each other. + // The [[ResolveJoinStrategyHints]] rule is applied before the resolution batch + // in the analyzer and we cannot semantically compare them at this stage. + // Therefore, we follow a simple rule; they match if an identifier in a hint + // is a tail of an identifier in a relation. This process is independent of a session + // catalog (`currentDb` in [[SessionCatalog]]) and it just compares them literally. + // + // For example, + // * in a query `SELECT /*+ BROADCAST(t) */ * FROM db1.t JOIN t`, + // the broadcast hint will match both tables, `db1.t` and `t`, + // even when the current db is `db2`. + // * in a query `SELECT /*+ BROADCAST(default.t) */ * FROM default.t JOIN t`, + // the broadcast hint will match the left-side table only, `default.t`. + private def matchedIdentifier(identInHint: Seq[String], identInQuery: Seq[String]): Boolean = { + if (identInHint.length <= identInQuery.length) { + identInHint.zip(identInQuery.takeRight(identInHint.length)) + .forall { case (i1, i2) => resolver(i1, i2) } + } else { + false + } + } + + private def extractIdentifier(r: SubqueryAlias): Seq[String] = { + r.identifier.qualifier :+ r.identifier.name + } + private def applyJoinStrategyHint( plan: LogicalPlan, - relations: mutable.HashSet[String], + relationsInHint: Set[Seq[String]], + relationsInHintWithMatch: mutable.HashSet[Seq[String]], hintName: String): LogicalPlan = { // Whether to continue recursing down the tree var recurse = true + def matchedIdentifierInHint(identInQuery: Seq[String]): Boolean = { + relationsInHint.find(matchedIdentifier(_, identInQuery)) + .map(relationsInHintWithMatch.add).nonEmpty + } + val newNode = CurrentOrigin.withOrigin(plan.origin) { plan match { case ResolvedHint(u @ UnresolvedRelation(ident), hint) - if relations.exists(resolver(_, ident.last)) => - relations.remove(ident.last) + if matchedIdentifierInHint(ident) => ResolvedHint(u, createHintInfo(hintName).merge(hint, hintErrorHandler)) case ResolvedHint(r: SubqueryAlias, hint) - if relations.exists(resolver(_, r.alias)) => - relations.remove(r.alias) + if matchedIdentifierInHint(extractIdentifier(r)) => ResolvedHint(r, createHintInfo(hintName).merge(hint, hintErrorHandler)) - case u @ UnresolvedRelation(ident) if relations.exists(resolver(_, ident.last)) => - relations.remove(ident.last) + case UnresolvedRelation(ident) if matchedIdentifierInHint(ident) => ResolvedHint(plan, createHintInfo(hintName)) - case r: SubqueryAlias if relations.exists(resolver(_, r.alias)) => - relations.remove(r.alias) + case r: SubqueryAlias if matchedIdentifierInHint(extractIdentifier(r)) => ResolvedHint(plan, createHintInfo(hintName)) case _: ResolvedHint | _: View | _: With | _: SubqueryAlias => @@ -107,7 +135,9 @@ object ResolveHints { } if ((plan fastEquals newNode) && recurse) { - newNode.mapChildren(child => applyJoinStrategyHint(child, relations, hintName)) + newNode.mapChildren { child => + applyJoinStrategyHint(child, relationsInHint, relationsInHintWithMatch, hintName) + } } else { newNode } @@ -120,17 +150,19 @@ object ResolveHints { ResolvedHint(h.child, createHintInfo(h.name)) } else { // Otherwise, find within the subtree query plans to apply the hint. - val relationNames = h.parameters.map { - case tableName: String => tableName - case tableId: UnresolvedAttribute => tableId.name + val relationNamesInHint = h.parameters.map { + case tableName: String => UnresolvedAttribute.parseAttributeName(tableName) + case tableId: UnresolvedAttribute => tableId.nameParts case unsupported => throw new AnalysisException("Join strategy hint parameter " + s"should be an identifier or string but was $unsupported (${unsupported.getClass}") - } - val relationNameSet = new mutable.HashSet[String] - relationNames.foreach(relationNameSet.add) - - val applied = applyJoinStrategyHint(h.child, relationNameSet, h.name) - hintErrorHandler.hintRelationsNotFound(h.name, h.parameters, relationNameSet.toSet) + }.toSet + val relationsInHintWithMatch = new mutable.HashSet[Seq[String]] + val applied = applyJoinStrategyHint( + h.child, relationNamesInHint, relationsInHintWithMatch, h.name) + + // Filters unmatched relation identifiers in the hint + val unmatchedIdents = relationNamesInHint -- relationsInHintWithMatch + hintErrorHandler.hintRelationsNotFound(h.name, h.parameters, unmatchedIdents) applied } } @@ -246,5 +278,4 @@ object ResolveHints { h.child } } - } 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 0a0bef6c9953a..eb9a4d4feb783 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 @@ -23,6 +23,7 @@ import scala.annotation.tailrec import scala.collection.mutable import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -63,6 +64,7 @@ object TypeCoercion { ImplicitTypeCasts :: DateTimeOperations :: WindowFrameCoercion :: + StringLiteralCoercion :: Nil // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. @@ -1043,6 +1045,34 @@ object TypeCoercion { } } } + + /** + * A special rule to support string literal as the second argument of date_add/date_sub functions, + * to keep backward compatibility as a temporary workaround. + * TODO(SPARK-28589): implement ANSI type type coercion and handle string literals. + */ + object StringLiteralCoercion extends TypeCoercionRule { + override protected def coerceTypes(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + case DateAdd(l, r) if r.dataType == StringType && r.foldable => + val days = try { + AnsiCast(r, IntegerType).eval().asInstanceOf[Int] + } catch { + case e: NumberFormatException => throw new AnalysisException( + "The second argument of 'date_add' function needs to be an integer.", cause = Some(e)) + } + DateAdd(l, Literal(days)) + case DateSub(l, r) if r.dataType == StringType && r.foldable => + val days = try { + AnsiCast(r, IntegerType).eval().asInstanceOf[Int] + } catch { + case e: NumberFormatException => throw new AnalysisException( + "The second argument of 'date_sub' function needs to be an integer.", cause = Some(e)) + } + DateSub(l, Literal(days)) + } + } } trait TypeCoercionRule extends Rule[LogicalPlan] with Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index dcc143982a4aa..db930cf7890e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -136,6 +136,8 @@ trait ExternalCatalog { def listTables(db: String, pattern: String): Seq[String] + def listViews(db: String, pattern: String): Seq[String] + /** * Loads data into a table. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogWithListener.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogWithListener.scala index 86113d3ec3ead..c2613ff74da4a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogWithListener.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogWithListener.scala @@ -154,6 +154,10 @@ class ExternalCatalogWithListener(delegate: ExternalCatalog) delegate.listTables(db, pattern) } + override def listViews(db: String, pattern: String): Seq[String] = { + delegate.listViews(db, pattern) + } + override def loadTable( db: String, table: String, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index abf69939dea13..31644a5ae4e35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -346,6 +346,12 @@ class InMemoryCatalog( StringUtils.filterPattern(listTables(db), pattern) } + override def listViews(db: String, pattern: String): Seq[String] = synchronized { + requireDbExists(db) + val views = catalog(db).tables.filter(_._2.table.tableType == CatalogTableType.VIEW).keySet + StringUtils.filterPattern(views.toSeq.sorted, pattern) + } + override def loadTable( db: String, table: String, 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 3a63aff493df5..b79857cdccd22 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 @@ -880,6 +880,25 @@ class SessionCatalog( } } + /** + * List all matching views in the specified database, including local temporary views. + */ + def listViews(db: String, pattern: String): Seq[TableIdentifier] = { + val dbName = formatDatabaseName(db) + val dbViews = if (dbName == globalTempViewManager.database) { + globalTempViewManager.listViewNames(pattern).map { name => + TableIdentifier(name, Some(globalTempViewManager.database)) + } + } else { + requireDbExists(dbName) + externalCatalog.listViews(dbName, pattern).map { name => + TableIdentifier(name, Some(dbName)) + } + } + + dbViews ++ listLocalTempViews(pattern) + } + /** * List all matching local temporary views. */ 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 6e965ef807d1c..d02776b5d86f8 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 @@ -521,8 +521,11 @@ object CatalogColumnStat extends Logging { val VERSION = 2 - private def getTimestampFormatter(): TimestampFormatter = { - TimestampFormatter(format = "yyyy-MM-dd HH:mm:ss.SSSSSS", zoneId = ZoneOffset.UTC) + private def getTimestampFormatter(isParsing: Boolean): TimestampFormatter = { + TimestampFormatter( + format = "yyyy-MM-dd HH:mm:ss.SSSSSS", + zoneId = ZoneOffset.UTC, + needVarLengthSecondFraction = isParsing) } /** @@ -535,7 +538,7 @@ object CatalogColumnStat extends Logging { case DateType => DateFormatter(ZoneOffset.UTC).parse(s) case TimestampType if version == 1 => DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf(s)) - case TimestampType => getTimestampFormatter().parse(s) + case TimestampType => getTimestampFormatter(isParsing = true).parse(s) case ByteType => s.toByte case ShortType => s.toShort case IntegerType => s.toInt @@ -558,7 +561,7 @@ object CatalogColumnStat extends Logging { def toExternalString(v: Any, colName: String, dataType: DataType): String = { val externalValue = dataType match { case DateType => DateFormatter(ZoneOffset.UTC).format(v.asInstanceOf[Int]) - case TimestampType => getTimestampFormatter().format(v.asInstanceOf[Long]) + case TimestampType => getTimestampFormatter(isParsing = false).format(v.asInstanceOf[Long]) case BooleanType | _: IntegralType | FloatType | DoubleType => v case _: DecimalType => v.asInstanceOf[Decimal].toJavaBigDecimal // This version of Spark does not use min/max for binary/string types so we ignore it. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala index c6a03183ab45e..a7c243537acb7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala @@ -34,7 +34,8 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { options.timestampFormat, options.zoneId, options.locale, - legacyFormat = FAST_DATE_FORMAT) + legacyFormat = FAST_DATE_FORMAT, + needVarLengthSecondFraction = true) private val decimalParser = if (options.locale == Locale.US) { // Special handling the default locale for backward compatibility diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 8892037e03a7d..9d09cab4fd482 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -26,6 +26,7 @@ import com.univocity.parsers.csv.{CsvParserSettings, CsvWriterSettings, Unescape import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy class CSVOptions( @transient val parameters: CaseInsensitiveMap[String], @@ -148,8 +149,12 @@ class CSVOptions( val dateFormat: String = parameters.getOrElse("dateFormat", DateFormatter.defaultPattern) - val timestampFormat: String = - parameters.getOrElse("timestampFormat", s"${DateFormatter.defaultPattern}'T'HH:mm:ss.SSSXXX") + val timestampFormat: String = parameters.getOrElse("timestampFormat", + if (SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY) { + s"${DateFormatter.defaultPattern}'T'HH:mm:ss.SSSXXX" + } else { + s"${DateFormatter.defaultPattern}'T'HH:mm:ss[.SSS][XXX]" + }) val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala index 00e3d49787db1..4990da2bf3797 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala @@ -46,7 +46,8 @@ class UnivocityGenerator( options.timestampFormat, options.zoneId, options.locale, - legacyFormat = FAST_DATE_FORMAT) + legacyFormat = FAST_DATE_FORMAT, + needVarLengthSecondFraction = false) private val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index dd8537b02935e..8e87a82769471 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -89,7 +89,8 @@ class UnivocityParser( options.timestampFormat, options.zoneId, options.locale, - legacyFormat = FAST_DATE_FORMAT) + legacyFormat = FAST_DATE_FORMAT, + needVarLengthSecondFraction = true) private val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, @@ -100,7 +101,8 @@ class UnivocityParser( // Retrieve the raw record string. private def getCurrentInput: UTF8String = { - UTF8String.fromString(tokenizer.getContext.currentParsedContent().stripLineEnd) + val currentContent = tokenizer.getContext.currentParsedContent() + if (currentContent == null) null else UTF8String.fromString(currentContent.stripLineEnd) } // This parser first picks some tokens from the input tokens, according to the required schema, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala index 65bb9a8c642b6..e6a4c8f1d3749 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.lang.reflect.{Method, Modifier} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.types._ @@ -55,7 +55,7 @@ import org.apache.spark.util.Utils case class CallMethodViaReflection(children: Seq[Expression]) extends Expression with CodegenFallback { - override def prettyName: String = "reflect" + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("reflect") override def checkInputDataTypes(): TypeCheckResult = { if (children.size < 2) { 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 7c4316fe08433..8d82956cc6f74 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 @@ -206,8 +206,13 @@ object Cast { case _ => false // overflow } + /** + * Returns `true` if casting non-nullable values from `from` type to `to` type + * may return null. Note that the caller side should take care of input nullability + * first and only call this method if the input is not nullable. + */ def forceNullable(from: DataType, to: DataType): Boolean = (from, to) match { - case (NullType, _) => true + case (NullType, _) => false // empty array or map case case (_, _) if from == to => false case (StringType, BinaryType) => false @@ -265,7 +270,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } } - override def nullable: Boolean = Cast.forceNullable(child.dataType, dataType) || child.nullable + override def nullable: Boolean = child.nullable || Cast.forceNullable(child.dataType, dataType) protected def ansiEnabled: Boolean diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 1599321982ce2..f29ece2e03b08 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Locale import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -258,7 +258,8 @@ abstract class Expression extends TreeNode[Expression] { * Returns a user-facing string representation of this expression's name. * This should usually match the name of the function in SQL. */ - def prettyName: String = nodeName.toLowerCase(Locale.ROOT) + def prettyName: String = + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(nodeName.toLowerCase(Locale.ROOT)) protected def flatArguments: Iterator[Any] = stringArgs.flatMap { case t: Iterable[_] => t 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 10f8ec9617d1b..1ac7ca676a876 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 @@ -17,11 +17,14 @@ package org.apache.spark.sql.catalyst.expressions +import scala.collection.mutable + import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.types.{AbstractDataType, DataType} +import org.apache.spark.sql.types.{AbstractDataType, AnyDataType, DataType} /** * User-defined function. @@ -31,14 +34,9 @@ import org.apache.spark.sql.types.{AbstractDataType, DataType} * null. Use boxed type or [[Option]] if you wanna do the null-handling yourself. * @param dataType Return type of function. * @param children The input expressions of this UDF. - * @param inputPrimitives The analyzer should be aware of Scala primitive types so as to make the - * UDF return null if there is any null input value of these types. On the - * other hand, Java UDFs can only have boxed types, thus this parameter will - * always be all false. - * @param inputTypes The expected input types of this UDF, used to perform type coercion. If we do - * 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 inputEncoders ExpressionEncoder for each input parameters. For a input parameter which + * serialized as struct will use encoder instead of CatalystTypeConverters to + * convert internal value to Scala value. * @param udfName The user-specified name of this UDF. * @param nullable True if the UDF can return null value. * @param udfDeterministic True if the UDF is deterministic. Deterministic UDF returns same result @@ -48,8 +46,7 @@ case class ScalaUDF( function: AnyRef, dataType: DataType, children: Seq[Expression], - inputPrimitives: Seq[Boolean], - inputTypes: Seq[AbstractDataType] = Nil, + inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil, udfName: Option[String] = None, nullable: Boolean = true, udfDeterministic: Boolean = true) @@ -59,6 +56,68 @@ case class ScalaUDF( override def toString: String = s"${udfName.getOrElse("UDF")}(${children.mkString(", ")})" + /** + * The analyzer should be aware of Scala primitive types so as to make the + * UDF return null if there is any null input value of these types. On the + * other hand, Java UDFs can only have boxed types, thus this will return + * Nil(has same effect with all false) and analyzer will skip null-handling + * on them. + */ + def inputPrimitives: Seq[Boolean] = { + inputEncoders.map { encoderOpt => + // It's possible that some of the inputs don't have a specific encoder(e.g. `Any`) + if (encoderOpt.isDefined) { + val encoder = encoderOpt.get + if (encoder.isSerializedAsStruct) { + // struct type is not primitive + false + } else { + // `nullable` is false iff the type is primitive + !encoder.schema.head.nullable + } + } else { + // Any type is not primitive + false + } + } + } + + /** + * The expected input types of this UDF, used to perform type coercion. If we do + * 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. + */ + def inputTypes: Seq[AbstractDataType] = { + inputEncoders.map { encoderOpt => + if (encoderOpt.isDefined) { + val encoder = encoderOpt.get + if (encoder.isSerializedAsStruct) { + encoder.schema + } else { + encoder.schema.head.dataType + } + } else { + AnyDataType + } + } + } + + private def createToScalaConverter(i: Int, dataType: DataType): Any => Any = { + if (inputEncoders.isEmpty) { + // for untyped Scala UDF + CatalystTypeConverters.createToScalaConverter(dataType) + } else { + val encoder = inputEncoders(i) + if (encoder.isDefined && encoder.get.isSerializedAsStructForTopLevel) { + val enc = encoder.get.resolveAndBind() + row: Any => enc.fromRow(row.asInstanceOf[InternalRow]) + } else { + CatalystTypeConverters.createToScalaConverter(dataType) + } + } + } + // scalastyle:off line.size.limit /** This method has been generated by this script @@ -66,7 +125,7 @@ case class ScalaUDF( (1 to 22).map { x => val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) val childs = (0 to x - 1).map(x => s"val child$x = children($x)").reduce(_ + "\n " + _) - val converters = (0 to x - 1).map(x => s"lazy val converter$x = CatalystTypeConverters.createToScalaConverter(child$x.dataType)").reduce(_ + "\n " + _) + val converters = (0 to x - 1).map(x => s"lazy val converter$x = createToScalaConverter($x, child$x.dataType)").reduce(_ + "\n " + _) val evals = (0 to x - 1).map(x => s"converter$x(child$x.eval(input))").reduce(_ + ",\n " + _) s"""case $x => @@ -91,7 +150,7 @@ case class ScalaUDF( case 1 => val func = function.asInstanceOf[(Any) => Any] val child0 = children(0) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) (input: InternalRow) => { func( converter0(child0.eval(input))) @@ -101,8 +160,8 @@ case class ScalaUDF( val func = function.asInstanceOf[(Any, Any) => Any] val child0 = children(0) val child1 = children(1) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -114,9 +173,9 @@ case class ScalaUDF( val child0 = children(0) val child1 = children(1) val child2 = children(2) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -130,10 +189,10 @@ case class ScalaUDF( val child1 = children(1) val child2 = children(2) val child3 = children(3) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -149,11 +208,11 @@ case class ScalaUDF( val child2 = children(2) val child3 = children(3) val child4 = children(4) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -171,12 +230,12 @@ case class ScalaUDF( val child3 = children(3) val child4 = children(4) val child5 = children(5) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -196,13 +255,13 @@ case class ScalaUDF( val child4 = children(4) val child5 = children(5) val child6 = children(6) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -224,14 +283,14 @@ case class ScalaUDF( val child5 = children(5) val child6 = children(6) val child7 = children(7) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -255,15 +314,15 @@ case class ScalaUDF( val child6 = children(6) val child7 = children(7) val child8 = children(8) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -289,16 +348,16 @@ case class ScalaUDF( val child7 = children(7) val child8 = children(8) val child9 = children(9) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -326,17 +385,17 @@ case class ScalaUDF( val child8 = children(8) val child9 = children(9) val child10 = children(10) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -366,18 +425,18 @@ case class ScalaUDF( val child9 = children(9) val child10 = children(10) val child11 = children(11) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -409,19 +468,19 @@ case class ScalaUDF( val child10 = children(10) val child11 = children(11) val child12 = children(12) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -455,20 +514,20 @@ case class ScalaUDF( val child11 = children(11) val child12 = children(12) val child13 = children(13) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) + lazy val converter13 = createToScalaConverter(13, child13.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -504,21 +563,21 @@ case class ScalaUDF( val child12 = children(12) val child13 = children(13) val child14 = children(14) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) + lazy val converter13 = createToScalaConverter(13, child13.dataType) + lazy val converter14 = createToScalaConverter(14, child14.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -556,22 +615,22 @@ case class ScalaUDF( val child13 = children(13) val child14 = children(14) val child15 = children(15) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) + lazy val converter13 = createToScalaConverter(13, child13.dataType) + lazy val converter14 = createToScalaConverter(14, child14.dataType) + lazy val converter15 = createToScalaConverter(15, child15.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -611,23 +670,23 @@ case class ScalaUDF( val child14 = children(14) val child15 = children(15) val child16 = children(16) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) + lazy val converter13 = createToScalaConverter(13, child13.dataType) + lazy val converter14 = createToScalaConverter(14, child14.dataType) + lazy val converter15 = createToScalaConverter(15, child15.dataType) + lazy val converter16 = createToScalaConverter(16, child16.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -669,24 +728,24 @@ case class ScalaUDF( val child15 = children(15) val child16 = children(16) val child17 = children(17) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) + lazy val converter13 = createToScalaConverter(13, child13.dataType) + lazy val converter14 = createToScalaConverter(14, child14.dataType) + lazy val converter15 = createToScalaConverter(15, child15.dataType) + lazy val converter16 = createToScalaConverter(16, child16.dataType) + lazy val converter17 = createToScalaConverter(17, child17.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -730,25 +789,25 @@ case class ScalaUDF( val child16 = children(16) val child17 = children(17) val child18 = children(18) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) + lazy val converter13 = createToScalaConverter(13, child13.dataType) + lazy val converter14 = createToScalaConverter(14, child14.dataType) + lazy val converter15 = createToScalaConverter(15, child15.dataType) + lazy val converter16 = createToScalaConverter(16, child16.dataType) + lazy val converter17 = createToScalaConverter(17, child17.dataType) + lazy val converter18 = createToScalaConverter(18, child18.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -794,26 +853,26 @@ case class ScalaUDF( val child17 = children(17) val child18 = children(18) val child19 = children(19) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) - lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) + lazy val converter13 = createToScalaConverter(13, child13.dataType) + lazy val converter14 = createToScalaConverter(14, child14.dataType) + lazy val converter15 = createToScalaConverter(15, child15.dataType) + lazy val converter16 = createToScalaConverter(16, child16.dataType) + lazy val converter17 = createToScalaConverter(17, child17.dataType) + lazy val converter18 = createToScalaConverter(18, child18.dataType) + lazy val converter19 = createToScalaConverter(19, child19.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -861,27 +920,27 @@ case class ScalaUDF( val child18 = children(18) val child19 = children(19) val child20 = children(20) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) - lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) - lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) + lazy val converter13 = createToScalaConverter(13, child13.dataType) + lazy val converter14 = createToScalaConverter(14, child14.dataType) + lazy val converter15 = createToScalaConverter(15, child15.dataType) + lazy val converter16 = createToScalaConverter(16, child16.dataType) + lazy val converter17 = createToScalaConverter(17, child17.dataType) + lazy val converter18 = createToScalaConverter(18, child18.dataType) + lazy val converter19 = createToScalaConverter(19, child19.dataType) + lazy val converter20 = createToScalaConverter(20, child20.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -931,28 +990,28 @@ case class ScalaUDF( val child19 = children(19) val child20 = children(20) val child21 = children(21) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) - lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) - lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) - lazy val converter21 = CatalystTypeConverters.createToScalaConverter(child21.dataType) + lazy val converter0 = createToScalaConverter(0, child0.dataType) + lazy val converter1 = createToScalaConverter(1, child1.dataType) + lazy val converter2 = createToScalaConverter(2, child2.dataType) + lazy val converter3 = createToScalaConverter(3, child3.dataType) + lazy val converter4 = createToScalaConverter(4, child4.dataType) + lazy val converter5 = createToScalaConverter(5, child5.dataType) + lazy val converter6 = createToScalaConverter(6, child6.dataType) + lazy val converter7 = createToScalaConverter(7, child7.dataType) + lazy val converter8 = createToScalaConverter(8, child8.dataType) + lazy val converter9 = createToScalaConverter(9, child9.dataType) + lazy val converter10 = createToScalaConverter(10, child10.dataType) + lazy val converter11 = createToScalaConverter(11, child11.dataType) + lazy val converter12 = createToScalaConverter(12, child12.dataType) + lazy val converter13 = createToScalaConverter(13, child13.dataType) + lazy val converter14 = createToScalaConverter(14, child14.dataType) + lazy val converter15 = createToScalaConverter(15, child15.dataType) + lazy val converter16 = createToScalaConverter(16, child16.dataType) + lazy val converter17 = createToScalaConverter(17, child17.dataType) + lazy val converter18 = createToScalaConverter(18, child18.dataType) + lazy val converter19 = createToScalaConverter(19, child19.dataType) + lazy val converter20 = createToScalaConverter(20, child20.dataType) + lazy val converter21 = createToScalaConverter(21, child21.dataType) (input: InternalRow) => { func( converter0(child0.eval(input)), @@ -987,8 +1046,8 @@ case class ScalaUDF( val converterClassName = classOf[Any => Any].getName // The type converters for inputs and the result. - val converters: Array[Any => Any] = children.map { c => - CatalystTypeConverters.createToScalaConverter(c.dataType) + val converters: Array[Any => Any] = children.zipWithIndex.map { case (c, i) => + createToScalaConverter(i, c.dataType) }.toArray :+ CatalystTypeConverters.createToCatalystConverter(dataType) val convertersTerm = ctx.addReferenceObj("converters", converters, s"$converterClassName[]") val errorMsgTerm = ctx.addReferenceObj("errMsg", udfErrorMessage) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala index 210acf33fc431..2c0060c22a865 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala @@ -117,7 +117,5 @@ case class First(child: Expression, ignoreNullsExpr: Expression) override lazy val evaluateExpression: AttributeReference = first - override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("first") - override def toString: String = s"$prettyName($child)${if (ignoreNulls) " ignore nulls"}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala index 2c89a4b973a7e..6793ac7632ffd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala @@ -115,7 +115,5 @@ case class Last(child: Expression, ignoreNullsExpr: Expression) override lazy val evaluateExpression: AttributeReference = last - override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("last") - override def toString: String = s"$prettyName($child)${if (ignoreNulls) " ignore nulls"}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 215e88a2cc8a4..6a64819aabb48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.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.{FunctionRegistry, TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{IntervalUtils, TypeUtils} @@ -457,6 +457,18 @@ case class Remainder(left: Expression, right: Expression) extends DivModLike { override def symbol: String = "%" override def decimalMethod: String = "remainder" + override def toString: String = { + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(sqlOperator) match { + case operator if operator == sqlOperator => s"($left $sqlOperator $right)" + case funcName => s"$funcName($left, $right)" + } + } + override def sql: String = { + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(sqlOperator) match { + case operator if operator == sqlOperator => s"(${left.sql} $sqlOperator ${right.sql})" + case funcName => s"$funcName(${left.sql}, ${right.sql})" + } + } private lazy val mod: (Any, Any) => Any = dataType match { // special cases to make float/double primitive types faster 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 58c95c94ba198..3042a27cdb838 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 @@ -45,7 +45,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.types._ -import org.apache.spark.util.{ParentClassLoader, Utils} +import org.apache.spark.util.{LongAccumulator, ParentClassLoader, Utils} /** * Java source for evaluating an [[Expression]] given a [[InternalRow]] of input. @@ -1309,6 +1309,23 @@ object CodeGenerator extends Logging { // bytecode instruction final val MUTABLESTATEARRAY_SIZE_LIMIT = 32768 + // The Java source code generated by whole-stage codegen on the Driver side is sent to each + // Executor for compilation and data processing. This is very effective in processing large + // amounts of data in a distributed environment. However, in the test environment, + // because the amount of data is not large or not executed in parallel, the compilation time + // of these Java source code will become a major part of the entire test runtime. When + // running test cases, we summarize the total compilation time and output it to the execution + // log for easy analysis and view. + private val _compileTime = new LongAccumulator + + // Returns the total compile time of Java source code in nanoseconds. + // Visible for testing + def compileTime: Long = _compileTime.sum + + // Reset compile time. + // Visible for testing + def resetCompileTime: Unit = _compileTime.reset() + /** * Compile the Java source code into a Java class, using Janino. * @@ -1466,10 +1483,12 @@ object CodeGenerator extends Logging { val startTime = System.nanoTime() val result = doCompile(code) val endTime = System.nanoTime() - def timeMs: Double = (endTime - startTime).toDouble / NANOS_PER_MILLIS + val duration = endTime - startTime + val timeMs: Double = duration.toDouble / NANOS_PER_MILLIS CodegenMetrics.METRIC_SOURCE_CODE_SIZE.update(code.body.length) CodegenMetrics.METRIC_COMPILATION_TIME.update(timeMs.toLong) logInfo(s"Code generated in $timeMs ms") + _compileTime.add(duration) result } }) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 6d95909e17f41..8b61bc4f22b94 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -77,9 +77,9 @@ trait BinaryArrayExpressionWithImplicitCast extends BinaryExpression @ExpressionDescription( usage = """ _FUNC_(expr) - Returns the size of an array or a map. - The function returns -1 if its input is null and spark.sql.legacy.sizeOfNull is set to true. - If spark.sql.legacy.sizeOfNull is set to false, the function returns null for null input. - By default, the spark.sql.legacy.sizeOfNull parameter is set to true. + The function returns null for null input if spark.sql.legacy.sizeOfNull is set to false or + spark.sql.ansi.enabled is set to true. Otherwise, the function returns -1 for null input. + With the default settings, the function returns -1 for null input. """, examples = """ Examples: 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 1e83a568995ea..44601b4b8db91 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 @@ -29,6 +29,7 @@ import org.apache.commons.text.StringEscapeUtils import org.apache.spark.SparkUpgradeException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{DateTimeUtils, LegacyDateFormats, TimestampFormatter} @@ -99,7 +100,8 @@ case class CurrentTimestamp() extends LeafExpression with CodegenFallback { override def eval(input: InternalRow): Any = currentTimestamp() - override def prettyName: String = "current_timestamp" + override def prettyName: String = + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("current_timestamp") } /** @@ -601,7 +603,7 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa arguments = """ Arguments: * timestamp - A date/timestamp or string to be converted to the given format. - * fmt - Date/time format pattern to follow. See `java.time.format.DateTimeFormatter` for valid date + * fmt - Date/time format pattern to follow. See Datetime Patterns for valid date and time format patterns. """, examples = """ @@ -626,14 +628,22 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti @transient private lazy val formatter: Option[TimestampFormatter] = { if (right.foldable) { Option(right.eval()).map { format => - TimestampFormatter(format.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) + TimestampFormatter( + format.toString, + zoneId, + legacyFormat = SIMPLE_DATE_FORMAT, + needVarLengthSecondFraction = false) } } else None } override protected def nullSafeEval(timestamp: Any, format: Any): Any = { val tf = if (formatter.isEmpty) { - TimestampFormatter(format.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) + TimestampFormatter( + format.toString, + zoneId, + legacyFormat = SIMPLE_DATE_FORMAT, + needVarLengthSecondFraction = false) } else { formatter.get } @@ -654,7 +664,8 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti s"""|UTF8String.fromString($tf$$.MODULE$$.apply( | $format.toString(), | $zid, - | $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT()) + | $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), + | false) |.format($timestamp))""".stripMargin }) } @@ -667,13 +678,14 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti * Converts time string with given pattern. * Deterministic version of [[UnixTimestamp]], must have at least one parameter. */ +// scalastyle:off line.size.limit @ExpressionDescription( usage = "_FUNC_(timeExp[, format]) - Returns the UNIX timestamp of the given time.", arguments = """ Arguments: * timeExp - A date/timestamp or string which is returned as a UNIX timestamp. * format - Date/time format pattern to follow. Ignored if `timeExp` is not a string. - Default value is "yyyy-MM-dd HH:mm:ss". See `java.time.format.DateTimeFormatter` + Default value is "yyyy-MM-dd HH:mm:ss". See Datetime Patterns for valid date and time format patterns. """, examples = """ @@ -682,6 +694,7 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti 1460098800 """, since = "1.6.0") +// scalastyle:on line.size.limit case class ToUnixTimestamp( timeExp: Expression, format: Expression, @@ -703,9 +716,10 @@ case class ToUnixTimestamp( override def prettyName: String = "to_unix_timestamp" } +// scalastyle:off line.size.limit /** * Converts time string with given pattern to Unix time stamp (in seconds), returns null if fail. - * See [https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html]. + * See Datetime Patterns. * Note that hive Language Manual says it returns 0 if fail, but in fact it returns null. * If the second parameter is missing, use "yyyy-MM-dd HH:mm:ss". * If no parameters provided, the first parameter will be current_timestamp. @@ -718,7 +732,7 @@ case class ToUnixTimestamp( Arguments: * timeExp - A date/timestamp or string. If not provided, this defaults to current time. * format - Date/time format pattern to follow. Ignored if `timeExp` is not a string. - Default value is "yyyy-MM-dd HH:mm:ss". See `java.time.format.DateTimeFormatter` + Default value is "yyyy-MM-dd HH:mm:ss". See Datetime Patterns for valid date and time format patterns. """, examples = """ @@ -729,6 +743,7 @@ case class ToUnixTimestamp( 1460041200 """, since = "1.5.0") +// scalastyle:on line.size.limit case class UnixTimestamp(timeExp: Expression, format: Expression, timeZoneId: Option[String] = None) extends UnixTime { @@ -767,7 +782,11 @@ abstract class ToTimestamp private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] private lazy val formatter: TimestampFormatter = try { - TimestampFormatter(constFormat.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) + TimestampFormatter( + constFormat.toString, + zoneId, + legacyFormat = SIMPLE_DATE_FORMAT, + needVarLengthSecondFraction = true) } catch { case NonFatal(_) => null } @@ -801,7 +820,11 @@ abstract class ToTimestamp } else { val formatString = f.asInstanceOf[UTF8String].toString try { - TimestampFormatter(formatString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) + TimestampFormatter( + formatString, + zoneId, + legacyFormat = SIMPLE_DATE_FORMAT, + needVarLengthSecondFraction = true) .parse(t.asInstanceOf[UTF8String].toString) / downScaleFactor } catch { case e: SparkUpgradeException => throw e @@ -850,7 +873,8 @@ abstract class ToTimestamp ${ev.value} = $tf$$.MODULE$$.apply( $format.toString(), $zid, - $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT()) + $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), + true) .parse($string.toString()) / $downScaleFactor; } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; @@ -897,12 +921,13 @@ abstract class UnixTime extends ToTimestamp { * format. If the format is missing, using format like "1970-01-01 00:00:00". * Note that hive Language Manual says it returns 0 if fail, but in fact it returns null. */ +// scalastyle:off line.size.limit @ExpressionDescription( usage = "_FUNC_(unix_time, format) - Returns `unix_time` in the specified `format`.", arguments = """ Arguments: * unix_time - UNIX Timestamp to be converted to the provided format. - * format - Date/time format pattern to follow. See `java.time.format.DateTimeFormatter` + * format - Date/time format pattern to follow. See Datetime Patterns for valid date and time format patterns. """, examples = """ @@ -911,6 +936,7 @@ abstract class UnixTime extends ToTimestamp { 1969-12-31 16:00:00 """, since = "1.5.0") +// scalastyle:on line.size.limit case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[String] = None) extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { @@ -936,7 +962,11 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] private lazy val formatter: TimestampFormatter = try { - TimestampFormatter(constFormat.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) + TimestampFormatter( + constFormat.toString, + zoneId, + legacyFormat = SIMPLE_DATE_FORMAT, + needVarLengthSecondFraction = false) } catch { case NonFatal(_) => null } @@ -963,7 +993,11 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ } else { try { UTF8String.fromString( - TimestampFormatter(f.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) + TimestampFormatter( + f.toString, + zoneId, + legacyFormat = SIMPLE_DATE_FORMAT, + needVarLengthSecondFraction = false) .format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) } catch { case NonFatal(_) => null @@ -1001,7 +1035,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ s""" try { ${ev.value} = UTF8String.fromString( - $tf$$.MODULE$$.apply($f.toString(), $zid, $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT()) + $tf$$.MODULE$$.apply($f.toString(), $zid, $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), false) .format($seconds * 1000000L)); } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; @@ -1423,6 +1457,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression) /** * Parses a column to a date based on the given format. */ +// scalastyle:off line.size.limit @ExpressionDescription( usage = """ _FUNC_(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to @@ -1432,7 +1467,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression) arguments = """ Arguments: * date_str - A string to be parsed to date. - * fmt - Date format pattern to follow. See `java.time.format.DateTimeFormatter` for valid + * fmt - Date format pattern to follow. See Datetime Patterns for valid date and time format patterns. """, examples = """ @@ -1443,6 +1478,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression) 2016-12-31 """, since = "1.5.0") +// scalastyle:on line.size.limit case class ParseToDate(left: Expression, format: Option[Expression], child: Expression) extends RuntimeReplaceable { @@ -1471,6 +1507,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr /** * Parses a column to a timestamp based on the supplied format. */ +// scalastyle:off line.size.limit @ExpressionDescription( usage = """ _FUNC_(timestamp_str[, fmt]) - Parses the `timestamp_str` expression with the `fmt` expression @@ -1480,7 +1517,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr arguments = """ Arguments: * timestamp_str - A string to be parsed to timestamp. - * fmt - Timestamp format pattern to follow. See `java.time.format.DateTimeFormatter` for valid + * fmt - Timestamp format pattern to follow. See Datetime Patterns for valid date and time format patterns. """, examples = """ @@ -1491,6 +1528,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr 2016-12-31 00:00:00 """, since = "2.2.0") +// scalastyle:on line.size.limit case class ParseToTimestamp(left: Expression, format: Option[Expression], child: Expression) extends RuntimeReplaceable { 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 4c2a511c11166..98068360183ff 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.io._ +import scala.collection.mutable.ArrayBuffer import scala.util.parsing.combinator.RegexParsers import com.fasterxml.jackson.core._ @@ -796,3 +797,139 @@ case class SchemaOfJson( override def prettyName: String = "schema_of_json" } + +/** + * A function that returns the number of elements in the outmost JSON array. + */ +@ExpressionDescription( + usage = "_FUNC_(jsonArray) - Returns the number of elements in the outmost JSON array.", + arguments = """ + Arguments: + * jsonArray - A JSON array. `NULL` is returned in case of any other valid JSON string, + `NULL` or an invalid JSON. + """, + examples = """ + Examples: + > SELECT _FUNC_('[1,2,3,4]'); + 4 + > SELECT _FUNC_('[1,2,3,{"f1":1,"f2":[5,6]},4]'); + 5 + > SELECT _FUNC_('[1,2'); + NULL + """, + since = "3.1.0" +) +case class LengthOfJsonArray(child: Expression) extends UnaryExpression + with CodegenFallback with ExpectsInputTypes { + + override def inputTypes: Seq[DataType] = Seq(StringType) + override def dataType: DataType = IntegerType + override def nullable: Boolean = true + override def prettyName: String = "json_array_length" + + override def eval(input: InternalRow): Any = { + val json = child.eval(input).asInstanceOf[UTF8String] + // return null for null input + if (json == null) { + return null + } + + try { + Utils.tryWithResource(CreateJacksonParser.utf8String(SharedFactory.jsonFactory, json)) { + parser => { + // return null if null array is encountered. + if (parser.nextToken() == null) { + return null + } + // Parse the array to compute its length. + parseCounter(parser, input) + } + } + } catch { + case _: JsonProcessingException | _: IOException => null + } + } + + private def parseCounter(parser: JsonParser, input: InternalRow): Any = { + var length = 0 + // Only JSON array are supported for this function. + if (parser.currentToken != JsonToken.START_ARRAY) { + return null + } + // Keep traversing until the end of JSON array + while(parser.nextToken() != JsonToken.END_ARRAY) { + length += 1 + // skip all the child of inner object or array + parser.skipChildren() + } + length + } +} + +/** + * A function which returns all the keys of the outmost JSON object. + */ +@ExpressionDescription( + usage = "_FUNC_(json_object) - Returns all the keys of the outmost JSON object as an array.", + arguments = """ + Arguments: + * json_object - A JSON object. If a valid JSON object is given, all the keys of the outmost + object will be returned as an array. If it is any other valid JSON string, an invalid JSON + string or an empty string, the function returns null. + """, + examples = """ + Examples: + > Select _FUNC_('{}'); + [] + > Select _FUNC_('{"key": "value"}'); + ["key"] + > Select _FUNC_('{"f1":"abc","f2":{"f3":"a", "f4":"b"}}'); + ["f1","f2"] + """, + since = "3.1.0" +) +case class JsonObjectKeys(child: Expression) extends UnaryExpression with CodegenFallback + with ExpectsInputTypes { + + override def inputTypes: Seq[DataType] = Seq(StringType) + override def dataType: DataType = ArrayType(StringType) + override def nullable: Boolean = true + override def prettyName: String = "json_object_keys" + + override def eval(input: InternalRow): Any = { + val json = child.eval(input).asInstanceOf[UTF8String] + // return null for `NULL` input + if(json == null) { + return null + } + + try { + Utils.tryWithResource(CreateJacksonParser.utf8String(SharedFactory.jsonFactory, json)) { + parser => { + // return null if an empty string or any other valid JSON string is encountered + if (parser.nextToken() == null || parser.currentToken() != JsonToken.START_OBJECT) { + return null + } + // Parse the JSON string to get all the keys of outmost JSON object + getJsonKeys(parser, input) + } + } + } catch { + case _: JsonProcessingException | _: IOException => null + } + } + + private def getJsonKeys(parser: JsonParser, input: InternalRow): GenericArrayData = { + var arrayBufferOfKeys = ArrayBuffer.empty[UTF8String] + + // traverse until the end of input and ensure it returns valid key + while(parser.nextValue() != null && parser.currentName() != null) { + // add current fieldName to the ArrayBuffer + arrayBufferOfKeys += UTF8String.fromString(parser.getCurrentName) + + // skip all the children of inner object or array + parser.skipChildren() + } + new GenericArrayData(arrayBufferOfKeys.toArray) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index d5b959b91c23d..66e6334e3a450 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -21,7 +21,7 @@ import java.{lang => jl} import java.util.Locale import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -62,8 +62,8 @@ abstract class UnaryMathExpression(val f: Double => Double, name: String) override def inputTypes: Seq[AbstractDataType] = Seq(DoubleType) override def dataType: DataType = DoubleType override def nullable: Boolean = true - override def toString: String = s"$name($child)" - override def prettyName: String = name + override def toString: String = s"$prettyName($child)" + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(name) protected override def nullSafeEval(input: Any): Any = { f(input.asInstanceOf[Double]) @@ -115,9 +115,9 @@ abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) override def inputTypes: Seq[DataType] = Seq(DoubleType, DoubleType) - override def toString: String = s"$name($left, $right)" + override def toString: String = s"$prettyName($left, $right)" - override def prettyName: String = name + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(name) override def dataType: DataType = DoubleType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index f576873829f27..8ce3ddd30a69e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -173,6 +173,7 @@ case class SparkVersion() extends LeafExpression with CodegenFallback { override def nullable: Boolean = false override def foldable: Boolean = true override def dataType: DataType = StringType + override def prettyName: String = "version" override def eval(input: InternalRow): Any = { UTF8String.fromString(SPARK_VERSION_SHORT + " " + SPARK_REVISION) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 02e90f8458c3d..77b4cecc263c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -382,6 +382,7 @@ case class OuterReference(e: NamedExpression) override def nullable: Boolean = e.nullable override def prettyName: String = "outer" + override def sql: String = s"$prettyName(${e.sql})" override def name: String = e.name override def qualifier: Seq[String] = e.qualifier override def exprId: ExprId = e.exprId diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 54abd09d89ddb..d5de95c65e49e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -28,7 +28,6 @@ import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.serializer._ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} -import org.apache.spark.sql.catalyst.ScalaReflection.universe.TermName import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ @@ -311,7 +310,7 @@ case class Invoke( override def nullable: Boolean = targetObject.nullable || needNullCheck || returnNullable override def children: Seq[Expression] = targetObject +: arguments - private lazy val encodedFunctionName = TermName(functionName).encodedName.toString + private lazy val encodedFunctionName = ScalaReflection.encodeFieldNameToIdentifier(functionName) @transient lazy val method = targetObject.dataType match { case ObjectType(cls) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala index fa2978cddcaae..f387e1774edbd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala @@ -59,15 +59,15 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends BaseOrdering { case dt: AtomicType if order.direction == Ascending => dt.ordering.asInstanceOf[Ordering[Any]].compare(left, right) case dt: AtomicType if order.direction == Descending => - dt.ordering.asInstanceOf[Ordering[Any]].reverse.compare(left, right) + - dt.ordering.asInstanceOf[Ordering[Any]].compare(left, right) case a: ArrayType if order.direction == Ascending => a.interpretedOrdering.asInstanceOf[Ordering[Any]].compare(left, right) case a: ArrayType if order.direction == Descending => - a.interpretedOrdering.asInstanceOf[Ordering[Any]].reverse.compare(left, right) + - a.interpretedOrdering.asInstanceOf[Ordering[Any]].compare(left, right) case s: StructType if order.direction == Ascending => s.interpretedOrdering.asInstanceOf[Ordering[Any]].compare(left, right) case s: StructType if order.direction == Descending => - s.interpretedOrdering.asInstanceOf[Ordering[Any]].reverse.compare(left, right) + - s.interpretedOrdering.asInstanceOf[Ordering[Any]].compare(left, right) case other => throw new IllegalArgumentException(s"Type $other does not support ordered operations") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 1b59056c4da66..8bf1f19844556 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -196,7 +196,7 @@ package object expressions { // For example, consider an example where "cat" is the catalog name, "db1" is the database // name, "a" is the table name and "b" is the column name and "c" is the struct field name. // If the name parts is cat.db1.a.b.c, then Attribute will match - // Attribute(b, qualifier("cat", "db1, "a")) and List("c") will be the second element + // Attribute(b, qualifier("cat", "db1", "a")) and List("c") will be the second element var matches: (Seq[Attribute], Seq[String]) = nameParts match { case catalogPart +: dbPart +: tblPart +: name +: nestedFields => val key = (catalogPart.toLowerCase(Locale.ROOT), dbPart.toLowerCase(Locale.ROOT), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index cc09f601db9c4..50a90ae40497a 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -27,7 +27,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.commons.codec.binary.{Base64 => CommonsBase64} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, TypeUtils} @@ -1450,7 +1450,7 @@ case class ParseUrl(children: Seq[Expression]) // scalastyle:on line.size.limit case class FormatString(children: Expression*) extends Expression with ImplicitCastInputTypes { - require(children.nonEmpty, "format_string() should take at least 1 argument") + require(children.nonEmpty, s"$prettyName() should take at least 1 argument") override def foldable: Boolean = children.forall(_.foldable) override def nullable: Boolean = children(0).nullable @@ -1517,7 +1517,8 @@ case class FormatString(children: Expression*) extends Expression with ImplicitC }""") } - override def prettyName: String = "format_string" + override def prettyName: String = getTagValue( + FunctionRegistry.FUNC_ALIAS).getOrElse("format_string") } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index e33cff2f14e17..f46a1c6836fcf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -105,24 +105,6 @@ object SubExprUtils extends PredicateHelper { e.find(_.isInstanceOf[OuterReference]).isDefined } - /** - * Returns whether there are any null-aware predicate subqueries inside Not. If not, we could - * turn the null-aware predicate into not-null-aware predicate. - */ - def hasNullAwarePredicateWithinNot(condition: Expression): Boolean = { - splitConjunctivePredicates(condition).exists { - case _: Exists | Not(_: Exists) => false - case _: InSubquery | Not(_: InSubquery) => false - case e => e.find { x => - x.isInstanceOf[Not] && e.find { - case _: InSubquery => true - case _ => false - }.isDefined - }.isDefined - } - - } - /** * Returns an expression after removing the OuterReference shell. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala index 073b45af51caf..55e06cb9e8471 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.xml -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback @@ -160,7 +160,8 @@ case class XPathFloat(xml: Expression, path: Expression) extends XPathExtract { """) // scalastyle:on line.size.limit case class XPathDouble(xml: Expression, path: Expression) extends XPathExtract { - override def prettyName: String = "xpath_double" + override def prettyName: String = + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("xpath_double") override def dataType: DataType = DoubleType override def nullSafeEval(xml: Any, path: Any): Any = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 45c4edff47070..f9222f5af54da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -27,6 +27,7 @@ import com.fasterxml.jackson.core.json.JsonReadFeature import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy /** * Options for parsing JSON data into Spark SQL rows. @@ -90,8 +91,12 @@ private[sql] class JSONOptions( val dateFormat: String = parameters.getOrElse("dateFormat", DateFormatter.defaultPattern) - val timestampFormat: String = - parameters.getOrElse("timestampFormat", s"${DateFormatter.defaultPattern}'T'HH:mm:ss.SSSXXX") + val timestampFormat: String = parameters.getOrElse("timestampFormat", + if (SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY) { + s"${DateFormatter.defaultPattern}'T'HH:mm:ss.SSSXXX" + } else { + s"${DateFormatter.defaultPattern}'T'HH:mm:ss[.SSS][XXX]" + }) val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index 141360ff02117..f3938feef0a35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -82,7 +82,8 @@ private[sql] class JacksonGenerator( options.timestampFormat, options.zoneId, options.locale, - legacyFormat = FAST_DATE_FORMAT) + legacyFormat = FAST_DATE_FORMAT, + needVarLengthSecondFraction = false) private val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index d0db06cae8161..8965a81feee19 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -60,7 +60,8 @@ class JacksonParser( options.timestampFormat, options.zoneId, options.locale, - legacyFormat = FAST_DATE_FORMAT) + legacyFormat = FAST_DATE_FORMAT, + needVarLengthSecondFraction = true) private val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index 3dd8694eae418..56b12784fd214 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -42,7 +42,8 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { options.timestampFormat, options.zoneId, options.locale, - legacyFormat = FAST_DATE_FORMAT) + legacyFormat = FAST_DATE_FORMAT, + needVarLengthSecondFraction = true) /** * Infer the type of a collection of json records in three stages: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index 9814bc247fc87..30da902a33cf2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -161,6 +161,59 @@ object NestedColumnAliasing { case MapType(keyType, valueType, _) => totalFieldNum(keyType) + totalFieldNum(valueType) case _ => 1 // UDT and others } +} + +/** + * This prunes unnessary nested columns from `Generate` and optional `Project` on top + * of it. + */ +object GeneratorNestedColumnAliasing { + def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match { + // Either `nestedPruningOnExpressions` or `nestedSchemaPruningEnabled` is enabled, we + // need to prune nested columns through Project and under Generate. The difference is + // when `nestedSchemaPruningEnabled` is on, nested columns will be pruned further at + // file format readers if it is supported. + case Project(projectList, g: Generate) if (SQLConf.get.nestedPruningOnExpressions || + SQLConf.get.nestedSchemaPruningEnabled) && canPruneGenerator(g.generator) => + // On top on `Generate`, a `Project` that might have nested column accessors. + // We try to get alias maps for both project list and generator's children expressions. + val exprsToPrune = projectList ++ g.generator.children + NestedColumnAliasing.getAliasSubMap(exprsToPrune, g.qualifiedGeneratorOutput).map { + case (nestedFieldToAlias, attrToAliases) => + val newChild = pruneGenerate(g, nestedFieldToAlias, attrToAliases) + Project(NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias), newChild) + } + + case g: Generate if SQLConf.get.nestedSchemaPruningEnabled && + canPruneGenerator(g.generator) => + // If any child output is required by higher projection, we cannot prune on it even we + // only use part of nested column of it. A required child output means it is referred + // as a whole or partially by higher projection, pruning it here will cause unresolved + // query plan. + NestedColumnAliasing.getAliasSubMap( + g.generator.children, g.requiredChildOutput).map { + case (nestedFieldToAlias, attrToAliases) => + pruneGenerate(g, nestedFieldToAlias, attrToAliases) + } + + case _ => + None + } + + private def pruneGenerate( + g: Generate, + nestedFieldToAlias: Map[ExtractValue, Alias], + attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = { + val newGenerator = g.generator.transform { + case f: ExtractValue if nestedFieldToAlias.contains(f) => + nestedFieldToAlias(f).toAttribute + }.asInstanceOf[Generator] + + // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. + val newGenerate = g.copy(generator = newGenerator) + + NestedColumnAliasing.replaceChildrenWithAliases(newGenerate, attrToAliases) + } /** * This is a while-list for pruning nested fields at `Generator`. 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 2ea3587165e10..4534794d8a374 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 @@ -609,24 +609,8 @@ object ColumnPruning extends Rule[LogicalPlan] { .map(_._2) p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices)) - // prune unrequired nested fields - case p @ Project(projectList, g: Generate) if SQLConf.get.nestedPruningOnExpressions && - NestedColumnAliasing.canPruneGenerator(g.generator) => - val exprsToPrune = projectList ++ g.generator.children - NestedColumnAliasing.getAliasSubMap(exprsToPrune, g.qualifiedGeneratorOutput).map { - case (nestedFieldToAlias, attrToAliases) => - val newGenerator = g.generator.transform { - case f: ExtractValue if nestedFieldToAlias.contains(f) => - nestedFieldToAlias(f).toAttribute - }.asInstanceOf[Generator] - - // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. - val newGenerate = g.copy(generator = newGenerator) - - val newChild = NestedColumnAliasing.replaceChildrenWithAliases(newGenerate, attrToAliases) - - Project(NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias), newChild) - }.getOrElse(p) + // prune unrequired nested fields from `Generate`. + case GeneratorNestedColumnAliasing(p) => p // Eliminate unneeded attributes from right side of a Left Existence Join. case j @ Join(_, right, LeftExistence(_), _, _) => @@ -1207,9 +1191,10 @@ object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelpe def getAliasMap(plan: Aggregate): AttributeMap[Expression] = { // Find all the aliased expressions in the aggregate list that don't include any actual - // AggregateExpression, and create a map from the alias to the expression + // AggregateExpression or PythonUDF, and create a map from the alias to the expression val aliasMap = plan.aggregateExpressions.collect { - case a: Alias if a.child.find(_.isInstanceOf[AggregateExpression]).isEmpty => + case a: Alias if a.child.find(e => e.isInstanceOf[AggregateExpression] || + PythonUDF.isGroupedAggPandasUDF(e)).isEmpty => (a.toAttribute, a.child) } AttributeMap(aliasMap) 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 b6974624c6514..6fdd2110ab12a 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 @@ -162,12 +162,33 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { plan: LogicalPlan): (Option[Expression], LogicalPlan) = { var newPlan = plan val newExprs = exprs.map { e => - e transformUp { + e transformDown { case Exists(sub, conditions, _) => val exists = AttributeReference("exists", BooleanType, nullable = false)() newPlan = buildJoin(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) exists + case Not(InSubquery(values, ListQuery(sub, conditions, _, _))) => + val exists = AttributeReference("exists", BooleanType, nullable = false)() + // Deduplicate conflicting attributes if any. + val newSub = dedupSubqueryOnSelfJoin(newPlan, sub, Some(values)) + val inConditions = values.zip(sub.output).map(EqualTo.tupled) + // To handle a null-aware predicate not-in-subquery in nested conditions + // (e.g., `v > 0 OR t1.id NOT IN (SELECT id FROM t2)`), we transform + // `inConditon` (t1.id=t2.id) into `(inCondition) OR ISNULL(inCondition)`. + // + // For example, `SELECT * FROM t1 WHERE v > 0 OR t1.id NOT IN (SELECT id FROM t2)` + // is transformed into a plan below; + // == Optimized Logical Plan == + // Project [id#78, v#79] + // +- Filter ((v#79 > 0) OR NOT exists#83) + // +- Join ExistenceJoin(exists#83), ((id#78 = id#80) OR isnull((id#78 = id#80))) + // :- Relation[id#78,v#79] parquet + // +- Relation[id#80] parquet + val nullAwareJoinConds = inConditions.map(c => Or(c, IsNull(c))) + val finalJoinCond = (nullAwareJoinConds ++ conditions).reduceLeft(And) + newPlan = Join(newPlan, newSub, ExistenceJoin(exists), Some(finalJoinCond), JoinHint.NONE) + Not(exists) case InSubquery(values, ListQuery(sub, conditions, _, _)) => val exists = AttributeReference("exists", BooleanType, nullable = false)() // Deduplicate conflicting attributes if any. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index b8d267009b548..d6a874d292d1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -104,6 +104,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging withOrigin(ctx)(StructType(visitColTypeList(ctx.colTypeList))) } + def parseRawDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) { + typedVisit[DataType](ctx.dataType()) + } + /* ******************************************************************************************** * Plan parsing * ******************************************************************************************** */ @@ -1545,12 +1549,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a Extract expression. */ override def visitExtract(ctx: ExtractContext): Expression = withOrigin(ctx) { - val fieldStr = ctx.field.getText - val source = expression(ctx.source) - val extractField = DatePart.parseExtractField(fieldStr, source, { - throw new ParseException(s"Literals of type '$fieldStr' are currently not supported.", ctx) - }) - new DatePart(Literal(fieldStr), expression(ctx.source), extractField) + val arguments = Seq(Literal(ctx.field.getText), expression(ctx.source)) + UnresolvedFunction("date_part", arguments, isDistinct = false) } /** @@ -2779,7 +2779,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case Some(query) => CreateTableAsSelectStatement( table, query, partitioning, bucketSpec, properties, provider, options, location, comment, - ifNotExists = ifNotExists) + writeOptions = Map.empty, ifNotExists = ifNotExists) case None if temp => // CREATE TEMPORARY TABLE ... USING ... is not supported by the catalyst parser. @@ -2834,7 +2834,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case Some(query) => ReplaceTableAsSelectStatement(table, query, partitioning, bucketSpec, properties, - provider, options, location, comment, orCreate = orCreate) + provider, options, location, comment, writeOptions = Map.empty, orCreate = orCreate) case _ => ReplaceTableStatement(table, schema.getOrElse(new StructType), partitioning, @@ -2897,6 +2897,16 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } + /** + * Create a [[ShowViews]] command. + */ + override def visitShowViews(ctx: ShowViewsContext): LogicalPlan = withOrigin(ctx) { + val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) + ShowViews( + UnresolvedNamespace(multiPart.getOrElse(Seq.empty[String])), + Option(ctx.pattern).map(string)) + } + override def visitColPosition(ctx: ColPositionContext): ColumnPosition = { ctx.position.getType match { case SqlBaseParser.FIRST => ColumnPosition.first() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index c96f2da9a5289..590193bddafb5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -40,6 +40,11 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log astBuilder.visitSingleDataType(parser.singleDataType()) } + /** Similar to `parseDataType`, but without CHAR/VARCHAR replacement. */ + override def parseRawDataType(sqlText: String): DataType = parse(sqlText) { parser => + astBuilder.parseRawDataType(parser.singleDataType()) + } + /** Creates Expression for a given SQL string. */ override def parseExpression(sqlText: String): Expression = parse(sqlText) { parser => astBuilder.visitSingleExpression(parser.singleExpression()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala index 77e357ad073da..d724933bc1029 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala @@ -70,4 +70,10 @@ trait ParserInterface { */ @throws[ParseException]("Text cannot be parsed to a DataType") def parseDataType(sqlText: String): DataType + + /** + * Parse a string to a raw [[DataType]] without CHAR/VARCHAR replacement. + */ + @throws[ParseException]("Text cannot be parsed to a DataType") + def parseRawDataType(sqlText: String): DataType } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 12482667efa0d..13e5b129765e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -23,6 +23,16 @@ import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode, TreeNodeTag import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} +/** + * An abstraction of the Spark SQL query plan tree, which can be logical or physical. This class + * defines some basic properties of a query plan node, as well as some new transform APIs to + * transform the expressions of the plan node. + * + * Note that, the query plan is a mutually recursive structure: + * QueryPlan -> Expression (subquery) -> QueryPlan + * The tree traverse APIs like `transform`, `foreach`, `collect`, etc. that are + * inherited from `TreeNode`, do not traverse into query plans inside subqueries. + */ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanType] { self: PlanType => @@ -133,7 +143,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT /** * Returns the result of running [[transformExpressions]] on this node - * and all its children. + * and all its children. Note that this method skips expressions inside subqueries. */ def transformAllExpressions(rule: PartialFunction[Expression, Expression]): this.type = { transform { @@ -204,7 +214,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT } /** - * All the subqueries of current plan. + * All the top-level subqueries of the current plan node. Nested subqueries are not included. */ def subqueries: Seq[PlanType] = { expressions.flatMap(_.collect { @@ -213,21 +223,21 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT } /** - * Returns a sequence containing the result of applying a partial function to all elements in this - * plan, also considering all the plans in its (nested) subqueries - */ - def collectInPlanAndSubqueries[B](f: PartialFunction[PlanType, B]): Seq[B] = - (this +: subqueriesAll).flatMap(_.collect(f)) - - /** - * Returns a sequence containing the subqueries in this plan, also including the (nested) - * subquries in its children + * All the subqueries of the current plan node and all its children. Nested subqueries are also + * included. */ def subqueriesAll: Seq[PlanType] = { val subqueries = this.flatMap(_.subqueries) subqueries ++ subqueries.flatMap(_.subqueriesAll) } + /** + * A variant of `collect`. This method not only apply the given function to all elements in this + * plan, also considering all the plans in its (nested) subqueries + */ + def collectWithSubqueries[B](f: PartialFunction[PlanType, B]): Seq[B] = + (this +: subqueriesAll).flatMap(_.collect(f)) + override def innerChildren: Seq[QueryPlan[_]] = subqueries /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala index f26e5662ee856..a325b61fcc5a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala @@ -186,7 +186,8 @@ trait HintErrorHandler { * @param parameters the hint parameters * @param invalidRelations the set of relation names that cannot be associated */ - def hintRelationsNotFound(name: String, parameters: Seq[Any], invalidRelations: Set[String]): Unit + def hintRelationsNotFound( + name: String, parameters: Seq[Any], invalidRelations: Set[Seq[String]]): Unit /** * Callback for a join hint specified on a relation that is not part of a join. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index aa1e9cc569916..b1129e741221b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -83,6 +83,7 @@ case class CreateTableAsSelectStatement( options: Map[String, String], location: Option[String], comment: Option[String], + writeOptions: Map[String, String], ifNotExists: Boolean) extends ParsedStatement { override def children: Seq[LogicalPlan] = Seq(asSelect) @@ -133,6 +134,7 @@ case class ReplaceTableAsSelectStatement( options: Map[String, String], location: Option[String], comment: Option[String], + writeOptions: Map[String, String], orCreate: Boolean) extends ParsedStatement { override def children: Seq[LogicalPlan] = Seq(asSelect) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index c04e56355a68f..579157a6f2f2e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -436,6 +436,22 @@ case class ShowTables( AttributeReference("tableName", StringType, nullable = false)()) } +/** + * The logical plan of the SHOW VIEWS command that works for v1 and v2 catalogs. + * + * Notes: v2 catalogs do not support views API yet, the command will fallback to + * v1 ShowViewsCommand during ResolveSessionCatalog. + */ +case class ShowViews( + namespace: LogicalPlan, + pattern: Option[String]) extends Command { + override def children: Seq[LogicalPlan] = Seq(namespace) + + override val output: Seq[Attribute] = Seq( + AttributeReference("namespace", StringType, nullable = false)(), + AttributeReference("viewName", StringType, nullable = false)()) +} + /** * The logical plan of the USE/USE NAMESPACE command that works for v2 catalogs. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index fab504174783e..05ec23f7ad479 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -51,12 +51,16 @@ trait DateTimeFormatterHelper { // In this way, synchronised is intentionally omitted in this method to make parallel calls // less synchronised. // The Cache.get method is not used here to avoid creation of additional instances of Callable. - protected def getOrCreateFormatter(pattern: String, locale: Locale): DateTimeFormatter = { - val newPattern = DateTimeUtils.convertIncompatiblePattern(pattern) - val key = (newPattern, locale) + protected def getOrCreateFormatter( + pattern: String, + locale: Locale, + needVarLengthSecondFraction: Boolean = false): DateTimeFormatter = { + val newPattern = convertIncompatiblePattern(pattern) + val useVarLen = needVarLengthSecondFraction && newPattern.contains('S') + val key = (newPattern, locale, useVarLen) var formatter = cache.getIfPresent(key) if (formatter == null) { - formatter = buildFormatter(newPattern, locale) + formatter = buildFormatter(newPattern, locale, useVarLen) cache.put(key, formatter) } formatter @@ -87,7 +91,9 @@ trait DateTimeFormatterHelper { private object DateTimeFormatterHelper { val cache = CacheBuilder.newBuilder() .maximumSize(128) - .build[(String, Locale), DateTimeFormatter]() + .build[(String, Locale, Boolean), DateTimeFormatter]() + + final val extractor = "^([^S]*)(S*)(.*)$".r def createBuilder(): DateTimeFormatterBuilder = { new DateTimeFormatterBuilder().parseCaseInsensitive() @@ -95,7 +101,6 @@ private object DateTimeFormatterHelper { def toFormatter(builder: DateTimeFormatterBuilder, locale: Locale): DateTimeFormatter = { builder - .parseDefaulting(ChronoField.ERA, 1) .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) @@ -105,8 +110,44 @@ private object DateTimeFormatterHelper { .withResolverStyle(ResolverStyle.STRICT) } - def buildFormatter(pattern: String, locale: Locale): DateTimeFormatter = { - val builder = createBuilder().appendPattern(pattern) + /** + * Building a formatter for parsing seconds fraction with variable length + */ + def createBuilderWithVarLengthSecondFraction( + pattern: String): DateTimeFormatterBuilder = { + val builder = createBuilder() + pattern.split("'").zipWithIndex.foreach { + // Split string starting with the regex itself which is `'` here will produce an extra empty + // string at res(0). So when the first element here is empty string we do not need append `'` + // literal to the DateTimeFormatterBuilder. + case ("", idx) if idx != 0 => builder.appendLiteral("'") + case (pattenPart, idx) if idx % 2 == 0 => + var rest = pattenPart + while (rest.nonEmpty) { + rest match { + case extractor(prefix, secondFraction, suffix) => + builder.appendPattern(prefix) + if (secondFraction.nonEmpty) { + builder.appendFraction(ChronoField.NANO_OF_SECOND, 1, secondFraction.length, false) + } + rest = suffix + case _ => throw new IllegalArgumentException(s"Unrecognized datetime pattern: $pattern") + } + } + case (patternPart, _) => builder.appendLiteral(patternPart) + } + builder + } + + def buildFormatter( + pattern: String, + locale: Locale, + varLenEnabled: Boolean): DateTimeFormatter = { + val builder = if (varLenEnabled) { + createBuilderWithVarLengthSecondFraction(pattern) + } else { + createBuilder().appendPattern(pattern) + } toFormatter(builder, locale) } @@ -120,4 +161,47 @@ private object DateTimeFormatterHelper { .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) toFormatter(builder, TimestampFormatter.defaultLocale) } + + final val unsupportedLetters = Set('A', 'c', 'e', 'n', 'N', 'p') + + /** + * In Spark 3.0, we switch to the Proleptic Gregorian calendar and use DateTimeFormatter for + * parsing/formatting datetime values. The pattern string is incompatible with the one defined + * by SimpleDateFormat in Spark 2.4 and earlier. This function converts all incompatible pattern + * for the new parser in Spark 3.0. See more details in SPARK-31030. + * @param pattern The input pattern. + * @return The pattern for new parser + */ + def convertIncompatiblePattern(pattern: String): String = { + val eraDesignatorContained = pattern.split("'").zipWithIndex.exists { + case (patternPart, index) => + // Text can be quoted using single quotes, we only check the non-quote parts. + index % 2 == 0 && patternPart.contains("G") + } + (pattern + " ").split("'").zipWithIndex.map { + case (patternPart, index) => + if (index % 2 == 0) { + for (c <- patternPart if unsupportedLetters.contains(c)) { + throw new IllegalArgumentException(s"Illegal pattern character: $c") + } + // The meaning of 'u' was day number of week in SimpleDateFormat, it was changed to year + // in DateTimeFormatter. Substitute 'u' to 'e' and use DateTimeFormatter to parse the + // string. If parsable, return the result; otherwise, fall back to 'u', and then use the + // legacy SimpleDateFormat parser to parse. When it is successfully parsed, throw an + // exception and ask users to change the pattern strings or turn on the legacy mode; + // otherwise, return NULL as what Spark 2.4 does. + val res = patternPart.replace("u", "e") + // In DateTimeFormatter, 'u' supports negative years. We substitute 'y' to 'u' here for + // keeping the support in Spark 3.0. If parse failed in Spark 3.0, fall back to 'y'. + // We only do this substitution when there is no era designator found in the pattern. + if (!eraDesignatorContained) { + res.replace("y", "u") + } else { + res + } + } else { + patternPart + } + }.mkString("'").stripSuffix(" ") + } } 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 9f207ec891e63..56259dfb114c2 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 @@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit._ import scala.util.control.NonFatal import org.apache.spark.sql.catalyst.util.DateTimeConstants._ +import org.apache.spark.sql.catalyst.util.RebaseDateTime._ import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -47,13 +48,6 @@ object DateTimeUtils { // it's 2440587.5, rounding up to compatible with Hive final val JULIAN_DAY_OF_EPOCH = 2440588 - final val GREGORIAN_CUTOVER_DAY = LocalDate.of(1582, 10, 15).toEpochDay - final val GREGORIAN_CUTOVER_MICROS = instantToMicros( - LocalDateTime.of(1582, 10, 15, 0, 0, 0) - .atOffset(ZoneOffset.UTC) - .toInstant) - final val GREGORIAN_CUTOVER_MILLIS = microsToMillis(GREGORIAN_CUTOVER_MICROS) - final val julianCommonEraStart = Timestamp.valueOf("0001-01-01 00:00:00") final val TimeZoneGMT = TimeZone.getTimeZone("GMT") @@ -92,53 +86,92 @@ object DateTimeUtils { } /** - * Returns the number of days since epoch from java.sql.Date. + * Converts an instance of `java.sql.Date` to a number of days since the epoch + * 1970-01-01 via extracting date fields `year`, `month`, `days` from the input, + * creating a local date in Proleptic Gregorian calendar from the fields, and + * getting the number of days from the resulted local date. + * + * This approach was taken to have the same local date as the triple of `year`, + * `month`, `day` in the original hybrid calendar used by `java.sql.Date` and + * Proleptic Gregorian calendar used by Spark since version 3.0.0, see SPARK-26651. + * + * @param date It represents a specific instant in time based on + * the hybrid calendar which combines Julian and + * Gregorian calendars. + * @return The number of days since epoch from java.sql.Date. */ def fromJavaDate(date: Date): SQLDate = { - if (date.getTime < GREGORIAN_CUTOVER_MILLIS) { - val era = if (date.before(julianCommonEraStart)) 0 else 1 - val localDate = date.toLocalDate.`with`(ChronoField.ERA, era) - localDateToDays(localDate) - } else { - microsToDays(millisToMicros(date.getTime)) - } + val millisUtc = date.getTime + val millisLocal = millisUtc + TimeZone.getDefault.getOffset(millisUtc) + val julianDays = Math.toIntExact(Math.floorDiv(millisLocal, MILLIS_PER_DAY)) + rebaseJulianToGregorianDays(julianDays) } /** - * Returns a java.sql.Date from number of days since epoch. + * The opposite to `fromJavaDate` method which converts a number of days to an + * instance of `java.sql.Date`. It builds a local date in Proleptic Gregorian + * calendar, extracts date fields `year`, `month`, `day`, and creates a local + * date in the hybrid calendar (Julian + Gregorian calendars) from the fields. + * + * The purpose of the conversion is to have the same local date as the triple + * of `year`, `month`, `day` in the original Proleptic Gregorian calendar and + * in the target calender. + * + * @param daysSinceEpoch The number of days since 1970-01-01. + * @return A `java.sql.Date` from number of days since epoch. */ def toJavaDate(daysSinceEpoch: SQLDate): Date = { - if (daysSinceEpoch < GREGORIAN_CUTOVER_DAY) { - Date.valueOf(LocalDate.ofEpochDay(daysSinceEpoch)) - } else { - new Date(microsToMillis(daysToMicros(daysSinceEpoch))) - } + val localDate = LocalDate.ofEpochDay(daysSinceEpoch) + new Date(localDate.getYear - 1900, localDate.getMonthValue - 1, localDate.getDayOfMonth) } /** - * Returns a java.sql.Timestamp from number of micros since epoch. + * Converts microseconds since the epoch to an instance of `java.sql.Timestamp` + * via creating a local timestamp at the system time zone in Proleptic Gregorian + * calendar, extracting date and time fields like `year` and `hours`, and forming + * new timestamp in the hybrid calendar from the extracted fields. + * + * The conversion is based on the JVM system time zone because the `java.sql.Timestamp` + * uses the time zone internally. + * + * The method performs the conversion via local timestamp fields to have the same date-time + * representation as `year`, `month`, `day`, ..., `seconds` in the original calendar + * and in the target calendar. + * + * @param us The number of microseconds since 1970-01-01T00:00:00.000000Z. + * @return A `java.sql.Timestamp` from number of micros since epoch. */ def toJavaTimestamp(us: SQLTimestamp): Timestamp = { - if (us < GREGORIAN_CUTOVER_MICROS) { - val ldt = microsToInstant(us).atZone(ZoneId.systemDefault()).toLocalDateTime - Timestamp.valueOf(ldt) - } else { - Timestamp.from(microsToInstant(us)) - } + val rebasedMicros = rebaseGregorianToJulianMicros(us) + val seconds = Math.floorDiv(rebasedMicros, MICROS_PER_SECOND) + val ts = new Timestamp(seconds * MILLIS_PER_SECOND) + val nanos = (rebasedMicros - seconds * MICROS_PER_SECOND) * NANOS_PER_MICROS + ts.setNanos(nanos.toInt) + ts } /** - * Returns the number of micros since epoch from java.sql.Timestamp. + * Converts an instance of `java.sql.Timestamp` to the number of microseconds since + * 1970-01-01T00:00:00.000000Z. It extracts date-time fields from the input, builds + * a local timestamp in Proleptic Gregorian calendar from the fields, and binds + * the timestamp to the system time zone. The resulted instant is converted to + * microseconds since the epoch. + * + * The conversion is performed via the system time zone because it is used internally + * in `java.sql.Timestamp` while extracting date-time fields. + * + * The goal of the function is to have the same local date-time in the original calendar + * - the hybrid calendar (Julian + Gregorian) and in the target calendar which is + * Proleptic Gregorian calendar, see SPARK-26651. + * + * @param t It represents a specific instant in time based on + * the hybrid calendar which combines Julian and + * Gregorian calendars. + * @return The number of micros since epoch from `java.sql.Timestamp`. */ def fromJavaTimestamp(t: Timestamp): SQLTimestamp = { - if (t.getTime < GREGORIAN_CUTOVER_MILLIS) { - val era = if (t.before(julianCommonEraStart)) 0 else 1 - val localDateTime = t.toLocalDateTime.`with`(ChronoField.ERA, era) - val instant = ZonedDateTime.of(localDateTime, ZoneId.systemDefault()).toInstant - instantToMicros(instant) - } else { - instantToMicros(t.toInstant) - } + val micros = millisToMicros(t.getTime) + (t.getNanos / NANOS_PER_MICROS) % MICROS_PER_MILLIS + rebaseJulianToGregorianMicros(micros) } /** @@ -148,7 +181,9 @@ object DateTimeUtils { def fromJulianDay(day: Int, nanoseconds: Long): SQLTimestamp = { // use Long to avoid rounding errors val seconds = (day - JULIAN_DAY_OF_EPOCH).toLong * SECONDS_PER_DAY - SECONDS.toMicros(seconds) + NANOSECONDS.toMicros(nanoseconds) + val micros = SECONDS.toMicros(seconds) + NANOSECONDS.toMicros(nanoseconds) + val rebased = rebaseJulianToGregorianMicros(micros) + rebased } /** @@ -157,7 +192,7 @@ object DateTimeUtils { * Note: support timestamp since 4717 BC (without negative nanoseconds, compatible with Hive). */ def toJulianDay(us: SQLTimestamp): (Int, Long) = { - val julian_us = us + JULIAN_DAY_OF_EPOCH * MICROS_PER_DAY + val julian_us = rebaseGregorianToJulianMicros(us) + JULIAN_DAY_OF_EPOCH * MICROS_PER_DAY val day = julian_us / MICROS_PER_DAY val micros = julian_us % MICROS_PER_DAY (day.toInt, MICROSECONDS.toNanos(micros)) @@ -936,42 +971,4 @@ object DateTimeUtils { val days = period.getDays new CalendarInterval(months, days, 0) } - - /** - * In Spark 3.0, we switch to the Proleptic Gregorian calendar and use DateTimeFormatter for - * parsing/formatting datetime values. The pattern string is incompatible with the one defined - * by SimpleDateFormat in Spark 2.4 and earlier. This function converts all incompatible pattern - * for the new parser in Spark 3.0. See more details in SPARK-31030. - * @param pattern The input pattern. - * @return The pattern for new parser - */ - def convertIncompatiblePattern(pattern: String): String = { - val eraDesignatorContained = pattern.split("'").zipWithIndex.exists { - case (patternPart, index) => - // Text can be quoted using single quotes, we only check the non-quote parts. - index % 2 == 0 && patternPart.contains("G") - } - pattern.split("'").zipWithIndex.map { - case (patternPart, index) => - if (index % 2 == 0) { - // The meaning of 'u' was day number of week in SimpleDateFormat, it was changed to year - // in DateTimeFormatter. Substitute 'u' to 'e' and use DateTimeFormatter to parse the - // string. If parsable, return the result; otherwise, fall back to 'u', and then use the - // legacy SimpleDateFormat parser to parse. When it is successfully parsed, throw an - // exception and ask users to change the pattern strings or turn on the legacy mode; - // otherwise, return NULL as what Spark 2.4 does. - val res = patternPart.replace("u", "e") - // In DateTimeFormatter, 'u' supports negative years. We substitute 'y' to 'u' here for - // keeping the support in Spark 3.0. If parse failed in Spark 3.0, fall back to 'y'. - // We only do this substitution when there is no era designator found in the pattern. - if (!eraDesignatorContained) { - res.replace("y", "u") - } else { - res - } - } else { - patternPart - } - }.mkString("'") - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala new file mode 100644 index 0000000000000..8ee6c87fae93a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -0,0 +1,426 @@ +/* + * 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.util + +import java.time.{LocalDate, LocalDateTime, ZoneId} +import java.time.temporal.ChronoField +import java.util.{Calendar, TimeZone} + +import scala.collection.mutable.AnyRefMap + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.{DefaultScalaModule, ScalaObjectMapper} + +import org.apache.spark.sql.catalyst.util.DateTimeConstants._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ + +/** + * The collection of functions for rebasing days and microseconds from/to the hybrid calendar + * (Julian + Gregorian since 1582-10-15) which is used by Spark 2.4 and earlier versions + * to/from Proleptic Gregorian calendar which is used by Spark since version 3.0. See SPARK-26651. + */ +object RebaseDateTime { + /** + * Rebases days since the epoch from an original to an target calendar, for instance, + * from a hybrid (Julian + Gregorian) to Proleptic Gregorian calendar. + * + * It finds the latest switch day which is less than the given `days`, and adds the difference + * in days associated with the switch days to the given `days`. + * The function is based on linear search which starts from the most recent switch days. + * This allows to perform less comparisons for modern dates. + * + * @param switches The days when difference in days between original and target calendar + * was changed. + * @param diffs The differences in days between calendars. + * @param days The number of days since the epoch 1970-01-01 to be rebased + * to the target calendar. + * @return The rebased days. + */ + private def rebaseDays(switches: Array[Int], diffs: Array[Int], days: Int): Int = { + var i = switches.length + do { i -= 1 } while (i > 0 && days < switches(i)) + days + diffs(i) + } + + // The differences in days between Julian and Proleptic Gregorian dates. + // The diff at the index `i` is applicable for all days in the date interval: + // [julianGregDiffSwitchDay(i), julianGregDiffSwitchDay(i+1)) + private val julianGregDiffs = Array(2, 1, 0, -1, -2, -3, -4, -5, -6, -7, -8, -9, -10, 0) + // The sorted days in Julian calendar when difference in days between Julian and + // Proleptic Gregorian calendars was changed. + // The starting point is the `0001-01-01` (-719164 days since the epoch in + // Julian calendar). This array is not applicable for dates before the staring point. + // Rebasing switch days and diffs `julianGregDiffSwitchDay` and `julianGregDiffs` + // was generated by the `localRebaseJulianToGregorianDays` function. + private val julianGregDiffSwitchDay = Array( + -719164, -682945, -646420, -609895, -536845, -500320, -463795, + -390745, -354220, -317695, -244645, -208120, -171595, -141427) + + // The first days of Common Era (CE) which is mapped to the '0001-01-01' date in Julian calendar. + private final val julianCommonEraStartDay = julianGregDiffSwitchDay(0) + + /** + * Converts the given number of days since the epoch day 1970-01-01 to a local date in Julian + * calendar, interprets the result as a local date in Proleptic Gregorian calendar, and takes the + * number of days since the epoch from the Gregorian local date. + * + * This is used to guarantee backward compatibility, as Spark 2.4 and earlier versions use + * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic Gregorian + * calendar. See SPARK-26651. + * + * For example: + * Julian calendar: 1582-01-01 -> -141704 + * Proleptic Gregorian calendar: 1582-01-01 -> -141714 + * The code below converts -141704 to -141714. + * + * @param days The number of days since the epoch in Julian calendar. It can be negative. + * @return The rebased number of days in Gregorian calendar. + */ + private[sql] def localRebaseJulianToGregorianDays(days: Int): Int = { + val utcCal = new Calendar.Builder() + // `gregory` is a hybrid calendar that supports both + // the Julian and Gregorian calendar systems + .setCalendarType("gregory") + .setTimeZone(TimeZoneUTC) + .setInstant(Math.multiplyExact(days, MILLIS_PER_DAY)) + .build() + val localDate = LocalDate.of( + utcCal.get(Calendar.YEAR), + utcCal.get(Calendar.MONTH) + 1, + // The number of days will be added later to handle non-existing + // Julian dates in Proleptic Gregorian calendar. + // For example, 1000-02-29 exists in Julian calendar because 1000 + // is a leap year but it is not a leap year in Gregorian calendar. + 1) + .`with`(ChronoField.ERA, utcCal.get(Calendar.ERA)) + .plusDays(utcCal.get(Calendar.DAY_OF_MONTH) - 1) + Math.toIntExact(localDate.toEpochDay) + } + + /** + * An optimized version of [[localRebaseJulianToGregorianDays(Int)]]. This method leverages the + * pre-calculated rebasing array to save calculation. For dates of Before Common Era, the + * function falls back to the regular unoptimized version. + * + * @param days The number of days since the epoch in Julian calendar. It can be negative. + * @return The rebased number of days in Gregorian calendar. + */ + def rebaseJulianToGregorianDays(days: Int): Int = { + if (days < julianCommonEraStartDay) { + localRebaseJulianToGregorianDays(days) + } else { + rebaseDays(julianGregDiffSwitchDay, julianGregDiffs, days) + } + } + + // The differences in days between Proleptic Gregorian and Julian dates. + // The diff at the index `i` is applicable for all days in the date interval: + // [gregJulianDiffSwitchDay(i), gregJulianDiffSwitchDay(i+1)) + private val gregJulianDiffs = Array(-2, -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0) + // The sorted days in Proleptic Gregorian calendar when difference in days between + // Proleptic Gregorian and Julian was changed. + // The starting point is the `0001-01-01` (-719162 days since the epoch in + // Proleptic Gregorian calendar). This array is not applicable for dates before the staring point. + // Rebasing switch days and diffs `gregJulianDiffSwitchDay` and `gregJulianDiffs` + // was generated by the `localRebaseGregorianToJulianDays` function. + private val gregJulianDiffSwitchDay = Array( + -719162, -682944, -646420, -609896, -536847, -500323, -463799, + -390750, -354226, -317702, -244653, -208129, -171605, -141427) + + // The first days of Common Era (CE) which is mapped to the '0001-01-01' date + // in Proleptic Gregorian calendar. + private final val gregorianCommonEraStartDay = gregJulianDiffSwitchDay(0) + + /** + * Converts the given number of days since the epoch day 1970-01-01 to a local date in Proleptic + * Gregorian calendar, interprets the result as a local date in Julian calendar, and takes the + * number of days since the epoch from the Julian local date. + * + * This is used to guarantee backward compatibility, as Spark 2.4 and earlier versions use + * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic Gregorian + * calendar. See SPARK-26651. + * + * For example: + * Proleptic Gregorian calendar: 1582-01-01 -> -141714 + * Julian calendar: 1582-01-01 -> -141704 + * The code below converts -141714 to -141704. + * + * @param days The number of days since the epoch in Proleptic Gregorian calendar. + * It can be negative. + * @return The rebased number of days in Julian calendar. + */ + private[sql] def localRebaseGregorianToJulianDays(days: Int): Int = { + val localDate = LocalDate.ofEpochDay(days) + val utcCal = new Calendar.Builder() + // `gregory` is a hybrid calendar that supports both + // the Julian and Gregorian calendar systems + .setCalendarType("gregory") + .setTimeZone(TimeZoneUTC) + .setDate(localDate.getYear, localDate.getMonthValue - 1, localDate.getDayOfMonth) + .build() + Math.toIntExact(Math.floorDiv(utcCal.getTimeInMillis, MILLIS_PER_DAY)) + } + + /** + * An optimized version of [[localRebaseGregorianToJulianDays(Int)]]. This method leverages the + * pre-calculated rebasing array to save calculation. For dates of Before Common Era, the + * function falls back to the regular unoptimized version. + * + * @param days The number of days since the epoch in Gregorian calendar. It can be negative. + * @return The rebased number of days since the epoch in Julian calendar. + */ + def rebaseGregorianToJulianDays(days: Int): Int = { + if (days < gregorianCommonEraStartDay) { + localRebaseGregorianToJulianDays(days) + } else { + rebaseDays(gregJulianDiffSwitchDay, gregJulianDiffs, days) + } + } + + + /** + * The class describes JSON records with microseconds rebasing info. + * Here is an example of JSON file: + * {{{ + * [ + * { + * "tz": "Europe/Paris", + * "switches": [-123, 0], + * "diffs": [422000000, 0] + * } + * ] + * }}} + * + * @param tz One of time zone ID which is expected to be acceptable by `ZoneId.of`. + * @param switches An ordered array of seconds since the epoch when the diff between + * two calendars are changed. + * @param diffs Differences in seconds associated with elements of `switches`. + */ + private case class JsonRebaseRecord(tz: String, switches: Array[Long], diffs: Array[Long]) + + /** + * Rebasing info used to convert microseconds from an original to a target calendar. + * + * @param switches An ordered array of microseconds since the epoch when the diff between + * two calendars are changed. + * @param diffs Differences in microseconds associated with elements of `switches`. + */ + private[sql] case class RebaseInfo(switches: Array[Long], diffs: Array[Long]) + + /** + * Rebases micros since the epoch from an original to an target calendar, for instance, + * from a hybrid (Julian + Gregorian) to Proleptic Gregorian calendar. + * + * It finds the latest switch micros which is less than the given `micros`, and adds the + * difference in micros associated with the switch micros to the given `micros`. + * The function is based on linear search which starts from the most recent switch micros. + * This allows to perform less comparisons for modern timestamps. + * + * @param rebaseInfo The rebasing info contains an ordered micros when difference in micros + * between original and target calendar was changed, + * and differences in micros between calendars + * @param micros The number of micros since the epoch 1970-01-01T00:00:00Z to be rebased + * to the target calendar. It can be negative. + * @return The rebased micros. + */ + private def rebaseMicros(rebaseInfo: RebaseInfo, micros: Long): Long = { + val switches = rebaseInfo.switches + var i = switches.length + do { i -= 1 } while (i > 0 && micros < switches(i)) + micros + rebaseInfo.diffs(i) + } + + // Loads rebasing info from an JSON file. JSON records in the files should conform to + // `JsonRebaseRecord`. AnyRefMap is used here instead of Scala's immutable map because + // it is 2 times faster in DateTimeRebaseBenchmark. + private[sql] def loadRebaseRecords(fileName: String): AnyRefMap[String, RebaseInfo] = { + val file = Thread.currentThread().getContextClassLoader.getResource(fileName) + val mapper = new ObjectMapper() with ScalaObjectMapper + mapper.registerModule(DefaultScalaModule) + val jsonRebaseRecords = mapper.readValue[Seq[JsonRebaseRecord]](file) + val anyRefMap = new AnyRefMap[String, RebaseInfo]((3 * jsonRebaseRecords.size) / 2) + jsonRebaseRecords.foreach { jsonRecord => + val rebaseInfo = RebaseInfo(jsonRecord.switches, jsonRecord.diffs) + var i = 0 + while (i < rebaseInfo.switches.length) { + rebaseInfo.switches(i) = rebaseInfo.switches(i) * MICROS_PER_SECOND + rebaseInfo.diffs(i) = rebaseInfo.diffs(i) * MICROS_PER_SECOND + i += 1 + } + anyRefMap.update(jsonRecord.tz, rebaseInfo) + } + anyRefMap + } + + /** + * A map of time zone IDs to its ordered time points (instants in microseconds since the epoch) + * when the difference between 2 instances associated with the same local timestamp in + * Proleptic Gregorian and the hybrid calendar was changed, and to the diff at the index `i` is + * applicable for all microseconds in the time interval: + * [gregJulianDiffSwitchMicros(i), gregJulianDiffSwitchMicros(i+1)) + */ + private val gregJulianRebaseMap = loadRebaseRecords("gregorian-julian-rebase-micros.json") + + /** + * Converts the given number of microseconds since the epoch '1970-01-01T00:00:00Z', to a local + * date-time in Proleptic Gregorian calendar with timezone identified by `zoneId`, interprets the + * result as a local date-time in Julian calendar with the same timezone, and takes microseconds + * since the epoch from the Julian local date-time. + * + * This is used to guarantee backward compatibility, as Spark 2.4 and earlier versions use + * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic Gregorian + * calendar. See SPARK-26651. + * + * For example: + * Proleptic Gregorian calendar: 1582-01-01 00:00:00.123456 -> -12244061221876544 + * Julian calendar: 1582-01-01 00:00:00.123456 -> -12243196799876544 + * The code below converts -12244061221876544 to -12243196799876544. + * + * @param zoneId The time zone ID at which the rebasing should be performed. + * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' + * in Proleptic Gregorian calendar. It can be negative. + * @return The rebased microseconds since the epoch in Julian calendar. + */ + private[sql] def rebaseGregorianToJulianMicros(zoneId: ZoneId, micros: Long): Long = { + val instant = microsToInstant(micros) + val ldt = instant.atZone(zoneId).toLocalDateTime + val cal = new Calendar.Builder() + // `gregory` is a hybrid calendar that supports both + // the Julian and Gregorian calendar systems + .setCalendarType("gregory") + .setDate(ldt.getYear, ldt.getMonthValue - 1, ldt.getDayOfMonth) + .setTimeOfDay(ldt.getHour, ldt.getMinute, ldt.getSecond) + // Local time-line can overlaps, such as at an autumn daylight savings cutover. + // This setting selects the original local timestamp mapped to the given `micros`. + .set(Calendar.DST_OFFSET, zoneId.getRules.getDaylightSavings(instant).toMillis.toInt) + .build() + millisToMicros(cal.getTimeInMillis) + ldt.get(ChronoField.MICRO_OF_SECOND) + } + + /** + * An optimized version of [[rebaseGregorianToJulianMicros(ZoneId, Long)]]. This method leverages + * the pre-calculated rebasing maps to save calculation. If the rebasing map doesn't contain + * information about the current JVM system time zone or `micros` is related to Before Common Era, + * the function falls back to the regular unoptimized version. + * + * Note: The function assumes that the input micros was derived from a local timestamp + * at the default system JVM time zone in Proleptic Gregorian calendar. + * + * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' + * in Proleptic Gregorian calendar. It can be negative. + * @return The rebased microseconds since the epoch in Julian calendar. + */ + def rebaseGregorianToJulianMicros(micros: Long): Long = { + val timeZone = TimeZone.getDefault + val tzId = timeZone.getID + val rebaseRecord = gregJulianRebaseMap.getOrNull(tzId) + if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { + rebaseGregorianToJulianMicros(timeZone.toZoneId, micros) + } else { + rebaseMicros(rebaseRecord, micros) + } + } + + /** + * Converts the given number of microseconds since the epoch '1970-01-01T00:00:00Z', to a local + * date-time in Julian calendar with timezone identified by `zoneId`, interprets the result as a + * local date-time in Proleptic Gregorian calendar with the same timezone, and takes microseconds + * since the epoch from the Gregorian local date-time. + * + * This is used to guarantee backward compatibility, as Spark 2.4 and earlier versions use + * Julian calendar for dates before 1582-10-15, while Spark 3.0 and later use Proleptic Gregorian + * calendar. See SPARK-26651. + * + * For example: + * Julian calendar: 1582-01-01 00:00:00.123456 -> -12243196799876544 + * Proleptic Gregorian calendar: 1582-01-01 00:00:00.123456 -> -12244061221876544 + * The code below converts -12243196799876544 to -12244061221876544. + * + * @param zoneId The time zone ID at which the rebasing should be performed. + * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' + * in the Julian calendar. It can be negative. + * @return The rebased microseconds since the epoch in Proleptic Gregorian calendar. + */ + private[sql] def rebaseJulianToGregorianMicros(zoneId: ZoneId, micros: Long): Long = { + val cal = new Calendar.Builder() + // `gregory` is a hybrid calendar that supports both + // the Julian and Gregorian calendar systems + .setCalendarType("gregory") + .setInstant(microsToMillis(micros)) + .build() + val localDateTime = LocalDateTime.of( + cal.get(Calendar.YEAR), + cal.get(Calendar.MONTH) + 1, + // The number of days will be added later to handle non-existing + // Julian dates in Proleptic Gregorian calendar. + // For example, 1000-02-29 exists in Julian calendar because 1000 + // is a leap year but it is not a leap year in Gregorian calendar. + 1, + cal.get(Calendar.HOUR_OF_DAY), + cal.get(Calendar.MINUTE), + cal.get(Calendar.SECOND), + (Math.floorMod(micros, MICROS_PER_SECOND) * NANOS_PER_MICROS).toInt) + .`with`(ChronoField.ERA, cal.get(Calendar.ERA)) + .plusDays(cal.get(Calendar.DAY_OF_MONTH) - 1) + val zonedDateTime = localDateTime.atZone(zoneId) + // Assuming the daylight saving switchover time is 2:00, the local clock will go back to + // 2:00 after hitting 2:59. This means the local time between [2:00, 3:00) appears twice, and + // can map to two different physical times (seconds from the UTC epoch). + // Java 8 time API resolves the ambiguity by picking the earlier physical time. This is the same + // as Java 7 time API, except for 2:00 where Java 7 picks the later physical time. + // Here we detect the "2:00" case and pick the latter physical time, to be compatible with the + // Java 7 date-time. + val adjustedZdt = if (cal.get(Calendar.DST_OFFSET) == 0) { + zonedDateTime.withLaterOffsetAtOverlap() + } else { + zonedDateTime + } + instantToMicros(adjustedZdt.toInstant) + } + + // The rebasing maps to convert microseconds from the hybrid calendar (Julian + Gregorian) + // to Proleptic Gregorian calendar. It maps time zone IDs to ordered timestamps (ascending order) + // where at every timestamps the difference between 2 calendars was changed, and to ordered + // differences between 2 calendars. The diff at the index `i` is applicable for all timestamps + // in the interval: [julianGregDiffSwitchMicros(i), julianGregDiffSwitchMicros(i+1)) + private val julianGregRebaseMap = loadRebaseRecords("julian-gregorian-rebase-micros.json") + + /** + * An optimized version of [[rebaseJulianToGregorianMicros(ZoneId, Long)]]. This method leverages + * the pre-calculated rebasing maps to save calculation. If the rebasing map doesn't contain + * information about the current JVM system time zone or `micros` is related to Before Common Era, + * the function falls back to the regular unoptimized version. + * + * Note: The function assumes that the input micros was derived from a local timestamp + * at the default system JVM time zone in the Julian calendar. + * + * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' + * in the Julian calendar. It can be negative. + * @return The rebased microseconds since the epoch in Proleptic Gregorian calendar. + */ + def rebaseJulianToGregorianMicros(micros: Long): Long = { + val timeZone = TimeZone.getDefault + val tzId = timeZone.getID + val rebaseRecord = julianGregRebaseMap.getOrNull(tzId) + if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { + rebaseJulianToGregorianMicros(timeZone.toZoneId, micros) + } else { + rebaseMicros(rebaseRecord, micros) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index f35a02ba7974c..611d6d65a999c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.text.{ParseException, ParsePosition, SimpleDateFormat} import java.time._ -import java.time.format.DateTimeParseException +import java.time.format.{DateTimeFormatter, DateTimeParseException} import java.time.temporal.ChronoField.MICRO_OF_SECOND import java.time.temporal.TemporalQueries import java.util.{Calendar, GregorianCalendar, Locale, TimeZone} @@ -55,10 +55,12 @@ class Iso8601TimestampFormatter( pattern: String, zoneId: ZoneId, locale: Locale, - legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT) + legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT, + needVarLengthSecondFraction: Boolean) extends TimestampFormatter with DateTimeFormatterHelper { @transient - protected lazy val formatter = getOrCreateFormatter(pattern, locale) + protected lazy val formatter: DateTimeFormatter = + getOrCreateFormatter(pattern, locale, needVarLengthSecondFraction) @transient protected lazy val legacyFormatter = TimestampFormatter.getLegacyFormatter( @@ -95,7 +97,8 @@ class Iso8601TimestampFormatter( * @param zoneId the time zone identifier in which the formatter parses or format timestamps */ class FractionTimestampFormatter(zoneId: ZoneId) - extends Iso8601TimestampFormatter("", zoneId, TimestampFormatter.defaultLocale) { + extends Iso8601TimestampFormatter( + "", zoneId, TimestampFormatter.defaultLocale, needVarLengthSecondFraction = false) { @transient override protected lazy val formatter = DateTimeFormatterHelper.fractionFormatter @@ -199,12 +202,14 @@ object TimestampFormatter { format: Option[String], zoneId: ZoneId, locale: Locale = defaultLocale, - legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT): TimestampFormatter = { + legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT, + needVarLengthSecondFraction: Boolean = false): TimestampFormatter = { val pattern = format.getOrElse(defaultPattern) if (SQLConf.get.legacyTimeParserPolicy == LEGACY) { getLegacyFormatter(pattern, zoneId, locale, legacyFormat) } else { - new Iso8601TimestampFormatter(pattern, zoneId, locale, legacyFormat) + new Iso8601TimestampFormatter( + pattern, zoneId, locale, legacyFormat, needVarLengthSecondFraction) } } @@ -227,16 +232,24 @@ object TimestampFormatter { format: String, zoneId: ZoneId, locale: Locale, - legacyFormat: LegacyDateFormat): TimestampFormatter = { - getFormatter(Some(format), zoneId, locale, legacyFormat) + legacyFormat: LegacyDateFormat, + needVarLengthSecondFraction: Boolean): TimestampFormatter = { + getFormatter(Some(format), zoneId, locale, legacyFormat, needVarLengthSecondFraction) } - def apply(format: String, zoneId: ZoneId, legacyFormat: LegacyDateFormat): TimestampFormatter = { - getFormatter(Some(format), zoneId, defaultLocale, legacyFormat) + def apply( + format: String, + zoneId: ZoneId, + legacyFormat: LegacyDateFormat, + needVarLengthSecondFraction: Boolean): TimestampFormatter = { + getFormatter(Some(format), zoneId, defaultLocale, legacyFormat, needVarLengthSecondFraction) } - def apply(format: String, zoneId: ZoneId): TimestampFormatter = { - getFormatter(Some(format), zoneId) + def apply( + format: String, + zoneId: ZoneId, + needVarLengthSecondFraction: Boolean = false): TimestampFormatter = { + getFormatter(Some(format), zoneId, needVarLengthSecondFraction = needVarLengthSecondFraction) } def apply(zoneId: ZoneId): TimestampFormatter = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 71bab624f06a8..d90804f4b6ff6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -20,7 +20,9 @@ package org.apache.spark.sql.connector.catalog import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.connector.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} +import org.apache.spark.sql.internal.SQLConf /** * Conversion helpers for working with v2 [[CatalogPlugin]]. @@ -132,4 +134,10 @@ private[sql] object CatalogV2Implicits { part } } + + private lazy val catalystSqlParser = new CatalystSqlParser(SQLConf.get) + + def parseColumnPath(name: String): Seq[String] = { + catalystSqlParser.parseMultipartIdentifier(name) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index ff63201b02b64..e1f329352592f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -22,11 +22,13 @@ import java.util.Collections import scala.collection.JavaConverters._ +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NamedRelation, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.AlterTable import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, DataType, HIVE_TYPE_STRING, HiveStringType, MapType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils @@ -329,4 +331,19 @@ private[sql] object CatalogV2Util { .getOrElse(catalogManager.v2SessionCatalog) .asTableCatalog } + + def failCharType(dt: DataType): Unit = { + if (HiveStringType.containsCharType(dt)) { + throw new AnalysisException( + "Cannot use CHAR type in non-Hive-Serde tables, please use STRING type instead.") + } + } + + def assertNoCharTypeInSchema(schema: StructType): Unit = { + schema.foreach { f => + if (f.metadata.contains(HIVE_TYPE_STRING)) { + failCharType(CatalystSqlParser.parseRawDataType(f.metadata.getString(HIVE_TYPE_STRING))) + } + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 777507954f791..bca75cc68ce65 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -490,12 +490,22 @@ object SQLConf { buildConf("spark.sql.adaptive.skewJoin.skewedPartitionFactor") .doc("A partition is considered as skewed if its size is larger than this factor " + "multiplying the median partition size and also larger than " + - s"'${ADVISORY_PARTITION_SIZE_IN_BYTES.key}'") + "'spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes'") .version("3.0.0") .intConf .checkValue(_ > 0, "The skew factor must be positive.") .createWithDefault(10) + val SKEW_JOIN_SKEWED_PARTITION_THRESHOLD = + buildConf("spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes") + .doc("A partition is considered as skewed if its size in bytes is larger than this " + + s"threshold and also larger than '${SKEW_JOIN_SKEWED_PARTITION_FACTOR.key}' " + + "multiplying the median partition size. Ideally this config should be set larger " + + s"than '${ADVISORY_PARTITION_SIZE_IN_BYTES.key}'.") + .version("3.0.0") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("256MB") + val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") .internal() @@ -1675,14 +1685,17 @@ object SQLConf { Try { DateTimeUtils.getZoneId(zone) }.isSuccess } - val SESSION_LOCAL_TIMEZONE = - buildConf("spark.sql.session.timeZone") - .doc("""The ID of session local timezone, e.g. "GMT", "America/Los_Angeles", etc.""") - .version("2.2.0") - .stringConf - .checkValue(isValidTimezone, s"Cannot resolve the given timezone with" + - " ZoneId.of(_, ZoneId.SHORT_IDS)") - .createWithDefaultFunction(() => TimeZone.getDefault.getID) + val SESSION_LOCAL_TIMEZONE = buildConf("spark.sql.session.timeZone") + .doc("The ID of session local timezone in the format of either region-based zone IDs or " + + "zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. " + + "Zone offsets must be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. " + + "Also 'UTC' and 'Z' are supported as aliases of '+00:00'. Other short names are not " + + "recommended to use because they can be ambiguous.") + .version("2.2.0") + .stringConf + .checkValue(isValidTimezone, s"Cannot resolve the given timezone with" + + " ZoneId.of(_, ZoneId.SHORT_IDS)") + .createWithDefaultFunction(() => TimeZone.getDefault.getID) val WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD = buildConf("spark.sql.windowExec.buffer.in.memory.threshold") @@ -1812,7 +1825,7 @@ object SQLConf { s"set, the fallback is `${BUFFER_SIZE.key}`. Note that Pandas execution requires more " + "than 4 bytes. Lowering this value could make small Pandas UDF batch iterated and " + "pipelined; however, it might degrade performance. See SPARK-27870.") - .version("3.1.0") + .version("3.0.0") .fallbackConf(BUFFER_SIZE) val PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME = @@ -2049,6 +2062,17 @@ object SQLConf { .booleanConf .createWithDefault(true) + val NESTED_PREDICATE_PUSHDOWN_ENABLED = + buildConf("spark.sql.optimizer.nestedPredicatePushdown.enabled") + .internal() + .doc("When true, Spark tries to push down predicates for nested columns and or names " + + "containing `dots` to data sources. Currently, Parquet implements both optimizations " + + "while ORC only supports predicates for names containing `dots`. The other data sources" + + "don't support this feature yet.") + .version("3.0.0") + .booleanConf + .createWithDefault(true) + val SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED = buildConf("spark.sql.optimizer.serializer.nestedSchemaPruning.enabled") .internal() @@ -2154,8 +2178,8 @@ object SQLConf { val LEGACY_SIZE_OF_NULL = buildConf("spark.sql.legacy.sizeOfNull") .internal() - .doc("If it is set to true, size of null returns -1. This behavior was inherited from Hive. " + - "The size function returns null for null input if the flag is disabled.") + .doc(s"If it is set to false, or ${ANSI_ENABLED.key} is true, then size of null returns " + + "null. Otherwise, it returns -1, which was inherited from Hive.") .version("2.4.0") .booleanConf .createWithDefault(true) @@ -2311,6 +2335,17 @@ object SQLConf { .booleanConf .createWithDefault(false) + val UI_EXPLAIN_MODE = buildConf("spark.sql.ui.explainMode") + .doc("Configures the query explain mode used in the Spark SQL UI. The value can be 'simple', " + + "'extended', 'codegen', 'cost', or 'formatted'. The default value is 'formatted'.") + .version("3.1.0") + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .checkValue(mode => Set("SIMPLE", "EXTENDED", "CODEGEN", "COST", "FORMATTED").contains(mode), + "Invalid value for 'spark.sql.ui.explainMode'. Valid values are 'simple', 'extended', " + + "'codegen', 'cost' and 'formatted'.") + .createWithDefault("formatted") + val SOURCES_BINARY_FILE_MAX_LENGTH = buildConf("spark.sql.sources.binaryFile.maxLength") .doc("The max length of a file that can be read by the binary file data source. " + "Spark will fail fast and not attempt to read the file if its length exceeds this value. " + @@ -2495,6 +2530,58 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_PARQUET_REBASE_DATETIME_IN_WRITE = + buildConf("spark.sql.legacy.parquet.rebaseDateTimeInWrite.enabled") + .internal() + .doc("When true, rebase dates/timestamps from Proleptic Gregorian calendar " + + "to the hybrid calendar (Julian + Gregorian) in write. " + + "The rebasing is performed by converting micros/millis/days to " + + "a local date/timestamp in the source calendar, interpreting the resulted date/" + + "timestamp in the target calendar, and getting the number of micros/millis/days " + + "since the epoch 1970-01-01 00:00:00Z.") + .version("3.0.0") + .booleanConf + .createWithDefault(false) + + val LEGACY_PARQUET_REBASE_DATETIME_IN_READ = + buildConf("spark.sql.legacy.parquet.rebaseDateTimeInRead.enabled") + .internal() + .doc("When true, rebase dates/timestamps " + + "from the hybrid calendar to Proleptic Gregorian calendar in read. " + + "The rebasing is performed by converting micros/millis/days to " + + "a local date/timestamp in the source calendar, interpreting the resulted date/" + + "timestamp in the target calendar, and getting the number of micros/millis/days " + + "since the epoch 1970-01-01 00:00:00Z.") + .version("3.0.0") + .booleanConf + .createWithDefault(false) + + val LEGACY_AVRO_REBASE_DATETIME_IN_WRITE = + buildConf("spark.sql.legacy.avro.rebaseDateTimeInWrite.enabled") + .internal() + .doc("When true, rebase dates/timestamps from Proleptic Gregorian calendar " + + "to the hybrid calendar (Julian + Gregorian) in write. " + + "The rebasing is performed by converting micros/millis/days to " + + "a local date/timestamp in the source calendar, interpreting the resulted date/" + + "timestamp in the target calendar, and getting the number of micros/millis/days " + + "since the epoch 1970-01-01 00:00:00Z.") + .version("3.0.0") + .booleanConf + .createWithDefault(false) + + val LEGACY_AVRO_REBASE_DATETIME_IN_READ = + buildConf("spark.sql.legacy.avro.rebaseDateTimeInRead.enabled") + .internal() + .doc("When true, rebase dates/timestamps " + + "from the hybrid calendar to Proleptic Gregorian calendar in read. " + + "The rebasing is performed by converting micros/millis/days to " + + "a local date/timestamp in the source calendar, interpreting the resulted date/" + + "timestamp in the target calendar, and getting the number of micros/millis/days " + + "since the epoch 1970-01-01 00:00:00Z.") + .version("3.0.0") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * @@ -2780,6 +2867,8 @@ class SQLConf extends Serializable with Logging { def datetimeJava8ApiEnabled: Boolean = getConf(DATETIME_JAVA8API_ENABLED) + def uiExplainMode: String = getConf(UI_EXPLAIN_MODE) + def addSingleFileInAddFile: Boolean = getConf(LEGACY_ADD_SINGLE_FILE_IN_ADD_FILE) def legacyMsSqlServerNumericMappingEnabled: Boolean = @@ -3015,6 +3104,8 @@ class SQLConf extends Serializable with Logging { def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) + def nestedPredicatePushdownEnabled: Boolean = getConf(NESTED_PREDICATE_PUSHDOWN_ENABLED) + def serializerNestedSchemaPruningEnabled: Boolean = getConf(SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED) @@ -3022,7 +3113,10 @@ class SQLConf extends Serializable with Logging { def csvColumnPruning: Boolean = getConf(SQLConf.CSV_PARSER_COLUMN_PRUNING) - def legacySizeOfNull: Boolean = getConf(SQLConf.LEGACY_SIZE_OF_NULL) + def legacySizeOfNull: Boolean = { + // size(null) should return null under ansi mode. + getConf(SQLConf.LEGACY_SIZE_OF_NULL) && !getConf(ANSI_ENABLED) + } def isReplEagerEvalEnabled: Boolean = getConf(SQLConf.REPL_EAGER_EVAL_ENABLED) @@ -3069,6 +3163,10 @@ class SQLConf extends Serializable with Logging { def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID) + def parquetRebaseDateTimeInReadEnabled: Boolean = { + getConf(SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_READ) + } + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 563e51ed597b2..d20252825c55b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -32,17 +32,20 @@ object StaticSQLConf { val WAREHOUSE_PATH = buildStaticConf("spark.sql.warehouse.dir") .doc("The default location for managed databases and tables.") + .version("2.0.0") .stringConf .createWithDefault(Utils.resolveURI("spark-warehouse").toString) val CATALOG_IMPLEMENTATION = buildStaticConf("spark.sql.catalogImplementation") .internal() + .version("2.0.0") .stringConf .checkValues(Set("hive", "in-memory")) .createWithDefault("in-memory") val GLOBAL_TEMP_DATABASE = buildStaticConf("spark.sql.globalTempDatabase") .internal() + .version("2.1.0") .stringConf .transform(_.toLowerCase(Locale.ROOT)) .createWithDefault("global_temp") @@ -55,9 +58,10 @@ object StaticSQLConf { // that's why this conf has to be a static SQL conf. val SCHEMA_STRING_LENGTH_THRESHOLD = buildStaticConf("spark.sql.sources.schemaStringLengthThreshold") + .internal() .doc("The maximum length allowed in a single cell when " + "storing additional schema information in Hive's metastore.") - .internal() + .version("1.3.1") .intConf .createWithDefault(4000) @@ -65,6 +69,7 @@ object StaticSQLConf { buildStaticConf("spark.sql.filesourceTableRelationCacheSize") .internal() .doc("The maximum size of the cache that maps qualified table names to table relation plans.") + .version("2.2.0") .intConf .checkValue(cacheSize => cacheSize >= 0, "The maximum size of the cache must not be negative") .createWithDefault(1000) @@ -73,6 +78,7 @@ object StaticSQLConf { .internal() .doc("When nonzero, enable caching of generated classes for operators and expressions. " + "All jobs share the cache that can use up to the specified number for generated classes.") + .version("2.4.0") .intConf .checkValue(maxEntries => maxEntries >= 0, "The maximum must not be negative") .createWithDefault(100) @@ -82,6 +88,7 @@ object StaticSQLConf { .doc("When true, put comment in the generated code. Since computing huge comments " + "can be extremely expensive in certain cases, such as deeply-nested expressions which " + "operate over inputs with wide schemas, default is false.") + .version("2.0.0") .booleanConf .createWithDefault(false) @@ -90,6 +97,7 @@ object StaticSQLConf { val DEBUG_MODE = buildStaticConf("spark.sql.debug") .internal() .doc("Only used for internal debugging. Not all functions are supported when it is enabled.") + .version("2.1.0") .booleanConf .createWithDefault(false) @@ -98,6 +106,7 @@ object StaticSQLConf { .doc("When set to true, Hive Thrift server is running in a single session mode. " + "All the JDBC/ODBC connections share the temporary views, function registries, " + "SQL configuration and the current database.") + .version("1.6.0") .booleanConf .createWithDefault(false) @@ -109,6 +118,7 @@ object StaticSQLConf { "applied in the specified order. For the case of parsers, the last parser is used and each " + "parser can delegate to its predecessor. For the case of function name conflicts, the last " + "registered function name is used.") + .version("2.2.0") .stringConf .toSequence .createOptional @@ -117,6 +127,7 @@ object StaticSQLConf { .doc("List of class names implementing QueryExecutionListener that will be automatically " + "added to newly created sessions. The classes should have either a no-arg constructor, " + "or a constructor that expects a SparkConf argument.") + .version("2.3.0") .stringConf .toSequence .createOptional @@ -125,6 +136,7 @@ object StaticSQLConf { .doc("List of class names implementing StreamingQueryListener that will be automatically " + "added to newly created sessions. The classes should have either a no-arg constructor, " + "or a constructor that expects a SparkConf argument.") + .version("2.4.0") .stringConf .toSequence .createOptional @@ -132,6 +144,7 @@ object StaticSQLConf { val UI_RETAINED_EXECUTIONS = buildStaticConf("spark.sql.ui.retainedExecutions") .doc("Number of executions to retain in the Spark UI.") + .version("1.5.0") .intConf .createWithDefault(1000) @@ -144,6 +157,7 @@ object StaticSQLConf { "Notice the number should be carefully chosen since decreasing parallelism might " + "cause longer waiting for other broadcasting. Also, increasing parallelism may " + "cause memory problem.") + .version("3.0.0") .intConf .checkValue(thres => thres > 0 && thres <= 128, "The threshold must be in (0,128].") .createWithDefault(128) @@ -152,6 +166,7 @@ object StaticSQLConf { buildStaticConf("spark.sql.subquery.maxThreadThreshold") .internal() .doc("The maximum degree of parallelism to execute the subquery.") + .version("2.4.6") .intConf .checkValue(thres => thres > 0 && thres <= 128, "The threshold must be in (0,128].") .createWithDefault(16) @@ -159,6 +174,7 @@ object StaticSQLConf { val SQL_EVENT_TRUNCATE_LENGTH = buildStaticConf("spark.sql.event.truncate.length") .doc("Threshold of SQL length beyond which it will be truncated before adding to " + "event. Defaults to no truncation. If set to 0, callsite will be logged instead.") + .version("3.0.0") .intConf .checkValue(_ >= 0, "Must be set greater or equal to zero") .createWithDefault(Int.MaxValue) @@ -167,11 +183,13 @@ object StaticSQLConf { buildStaticConf("spark.sql.legacy.sessionInitWithConfigDefaults") .doc("Flag to revert to legacy behavior where a cloned SparkSession receives SparkConf " + "defaults, dropping any overrides in its parent SparkSession.") + .version("3.0.0") .booleanConf .createWithDefault(false) val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED = buildStaticConf("spark.sql.defaultUrlStreamHandlerFactory.enabled") + .internal() .doc( "When true, register Hadoop's FsUrlStreamHandlerFactory to support " + "ADD JAR against HDFS locations. " + @@ -179,7 +197,7 @@ object StaticSQLConf { "to support a particular protocol type, or if Hadoop's FsUrlStreamHandlerFactory " + "conflicts with other protocol types such as `http` or `https`. See also SPARK-25694 " + "and HADOOP-14598.") - .internal() + .version("3.0.0") .booleanConf .createWithDefault(true) @@ -187,6 +205,7 @@ object StaticSQLConf { buildStaticConf("spark.sql.streaming.ui.enabled") .doc("Whether to run the Structured Streaming Web UI for the Spark application when the " + "Spark Web UI is enabled.") + .version("3.0.0") .booleanConf .createWithDefault(true) @@ -194,12 +213,14 @@ object StaticSQLConf { buildStaticConf("spark.sql.streaming.ui.retainedProgressUpdates") .doc("The number of progress updates to retain for a streaming query for Structured " + "Streaming UI.") + .version("3.0.0") .intConf .createWithDefault(100) val STREAMING_UI_RETAINED_QUERIES = buildStaticConf("spark.sql.streaming.ui.retainedQueries") .doc("The number of inactive queries to retain for Structured Streaming UI.") + .version("3.0.0") .intConf .createWithDefault(100) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala index 020dd79f8f0d7..319073e4475be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import org.apache.spark.annotation.{Evolving, Stable} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines all the filters that we can push down to the data sources. @@ -32,6 +33,10 @@ import org.apache.spark.annotation.{Evolving, Stable} sealed abstract class Filter { /** * List of columns that are referenced by this filter. + * + * Note that, each element in `references` represents a column; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, it is quoted to avoid confusion. + * * @since 2.1.0 */ def references: Array[String] @@ -40,12 +45,32 @@ sealed abstract class Filter { case f: Filter => f.references case _ => Array.empty } + + /** + * List of columns that are referenced by this filter. + * + * @return each element is a column name as an array of string multi-identifier + * @since 3.0.0 + */ + def v2references: Array[Array[String]] = { + this.references.map(parseColumnPath(_).toArray) + } + + /** + * If any of the references of this filter contains nested column + */ + private[sql] def containsNestedColumn: Boolean = { + this.v2references.exists(_.length > 1) + } } /** - * A filter that evaluates to `true` iff the attribute evaluates to a value + * A filter that evaluates to `true` iff the column evaluates to a value * equal to `value`. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.0 */ @Stable @@ -58,6 +83,9 @@ case class EqualTo(attribute: String, value: Any) extends Filter { * in that it returns `true` (rather than NULL) if both inputs are NULL, and `false` * (rather than NULL) if one of the input is NULL and the other is not NULL. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.5.0 */ @Stable @@ -69,6 +97,9 @@ case class EqualNullSafe(attribute: String, value: Any) extends Filter { * A filter that evaluates to `true` iff the attribute evaluates to a value * greater than `value`. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.0 */ @Stable @@ -80,6 +111,9 @@ case class GreaterThan(attribute: String, value: Any) extends Filter { * A filter that evaluates to `true` iff the attribute evaluates to a value * greater than or equal to `value`. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.0 */ @Stable @@ -91,6 +125,9 @@ case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter { * A filter that evaluates to `true` iff the attribute evaluates to a value * less than `value`. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.0 */ @Stable @@ -102,6 +139,9 @@ case class LessThan(attribute: String, value: Any) extends Filter { * A filter that evaluates to `true` iff the attribute evaluates to a value * less than or equal to `value`. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.0 */ @Stable @@ -112,6 +152,9 @@ case class LessThanOrEqual(attribute: String, value: Any) extends Filter { /** * A filter that evaluates to `true` iff the attribute evaluates to one of the values in the array. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.0 */ @Stable @@ -139,6 +182,9 @@ case class In(attribute: String, values: Array[Any]) extends Filter { /** * A filter that evaluates to `true` iff the attribute evaluates to null. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.0 */ @Stable @@ -149,6 +195,9 @@ case class IsNull(attribute: String) extends Filter { /** * A filter that evaluates to `true` iff the attribute evaluates to a non-null value. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.0 */ @Stable @@ -190,6 +239,9 @@ case class Not(child: Filter) extends Filter { * A filter that evaluates to `true` iff the attribute evaluates to * a string that starts with `value`. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.1 */ @Stable @@ -201,6 +253,9 @@ case class StringStartsWith(attribute: String, value: String) extends Filter { * A filter that evaluates to `true` iff the attribute evaluates to * a string that ends with `value`. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.1 */ @Stable @@ -212,6 +267,9 @@ case class StringEndsWith(attribute: String, value: String) extends Filter { * A filter that evaluates to `true` iff the attribute evaluates to * a string that contains the string `value`. * + * @param attribute of the column to be evaluated; `dots` are used as separators + * for nested columns. If any part of the names contains `dots`, + * it is quoted to avoid confusion. * @since 1.3.1 */ @Stable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/HiveStringType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/HiveStringType.scala index 4eb3226c5786e..a29f49ad14a77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/HiveStringType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/HiveStringType.scala @@ -53,6 +53,13 @@ object HiveStringType { case _: HiveStringType => StringType case _ => dt } + + def containsCharType(dt: DataType): Boolean = dt match { + case ArrayType(et, _) => containsCharType(et) + case MapType(kt, vt, _) => containsCharType(kt) || containsCharType(vt) + case StructType(fields) => fields.exists(f => containsCharType(f.dataType)) + case _ => dt.isInstanceOf[CharType] + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index f9cd9c3c398f6..b981a50499bf5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, If, SpecificInternalRow, UpCast} import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, NewInstance} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval case class PrimitiveData( intField: Int, @@ -48,7 +49,8 @@ case class NullableData( decimalField: java.math.BigDecimal, dateField: Date, timestampField: Timestamp, - binaryField: Array[Byte]) + binaryField: Array[Byte], + intervalField: CalendarInterval) case class OptionalData( intField: Option[Int], @@ -58,7 +60,8 @@ case class OptionalData( shortField: Option[Short], byteField: Option[Byte], booleanField: Option[Boolean], - structField: Option[PrimitiveData]) + structField: Option[PrimitiveData], + intervalField: Option[CalendarInterval]) case class ComplexData( arrayField: Seq[Int], @@ -200,7 +203,8 @@ class ScalaReflectionSuite extends SparkFunSuite { StructField("decimalField", DecimalType.SYSTEM_DEFAULT, nullable = true), StructField("dateField", DateType, nullable = true), StructField("timestampField", TimestampType, nullable = true), - StructField("binaryField", BinaryType, nullable = true))), + StructField("binaryField", BinaryType, nullable = true), + StructField("intervalField", CalendarIntervalType, nullable = true))), nullable = true)) } @@ -215,7 +219,8 @@ class ScalaReflectionSuite extends SparkFunSuite { StructField("shortField", ShortType, nullable = true), StructField("byteField", ByteType, nullable = true), StructField("booleanField", BooleanType, nullable = true), - StructField("structField", schemaFor[PrimitiveData].dataType, nullable = true))), + StructField("structField", schemaFor[PrimitiveData].dataType, nullable = true), + StructField("intervalField", CalendarIntervalType, nullable = true))), nullable = true)) } @@ -295,10 +300,10 @@ class ScalaReflectionSuite extends SparkFunSuite { test("convert Option[Product] to catalyst") { val primitiveData = PrimitiveData(1, 1, 1, 1, 1, 1, true) val data = OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true), - Some(primitiveData)) + Some(primitiveData), Some(new CalendarInterval(1, 2, 3))) val dataType = schemaFor[OptionalData].dataType val convertedData = InternalRow(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 2.toByte, true, - InternalRow(1, 1, 1, 1, 1, 1, true)) + InternalRow(1, 1, 1, 1, 1, 1, true), new CalendarInterval(1, 2, 3)) assert(CatalystTypeConverters.createToCatalystConverter(dataType)(data) === convertedData) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 09e0d9c4882fb..166ffec44a60d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -614,21 +614,6 @@ class AnalysisErrorSuite extends AnalysisTest { " in Filter" :: Nil) } - test("PredicateSubQuery is used is a nested condition") { - val a = AttributeReference("a", IntegerType)() - val b = AttributeReference("b", IntegerType)() - val c = AttributeReference("c", BooleanType)() - val plan1 = Filter(Cast(Not(InSubquery(Seq(a), ListQuery(LocalRelation(b)))), BooleanType), - LocalRelation(a)) - assertAnalysisError(plan1, - "Null-aware predicate sub-queries cannot be used in nested conditions" :: Nil) - - val plan2 = Filter( - Or(Not(InSubquery(Seq(a), ListQuery(LocalRelation(b)))), c), LocalRelation(a, c)) - assertAnalysisError(plan2, - "Null-aware predicate sub-queries cannot be used in nested conditions" :: Nil) - } - test("PredicateSubQuery correlated predicate is nested in an illegal plan") { val a = AttributeReference("a", IntegerType)() val b = AttributeReference("b", IntegerType)() 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 8451b9b50eff3..02472e153b09e 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 @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Count, Sum} @@ -326,20 +327,21 @@ class AnalysisSuite extends AnalysisTest with Matchers { } // non-primitive parameters do not need special null handling - val udf1 = ScalaUDF((s: String) => "x", StringType, string :: Nil, false :: Nil) + val udf1 = ScalaUDF((s: String) => "x", StringType, string :: Nil, + Option(ExpressionEncoder[String]()) :: Nil) val expected1 = udf1 checkUDF(udf1, expected1) // only primitive parameter needs special null handling val udf2 = ScalaUDF((s: String, d: Double) => "x", StringType, string :: double :: Nil, - false :: true :: Nil) + Option(ExpressionEncoder[String]()) :: Option(ExpressionEncoder[Double]()) :: Nil) val expected2 = If(IsNull(double), nullResult, udf2.copy(children = string :: KnownNotNull(double) :: Nil)) checkUDF(udf2, expected2) // special null handling should apply to all primitive parameters val udf3 = ScalaUDF((s: Short, d: Double) => "x", StringType, short :: double :: Nil, - true :: true :: Nil) + Option(ExpressionEncoder[Short]()) :: Option(ExpressionEncoder[Double]()) :: Nil) val expected3 = If( IsNull(short) || IsNull(double), nullResult, @@ -351,7 +353,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { (s: Short, d: Double) => "x", StringType, short :: nonNullableDouble :: Nil, - true :: true :: Nil) + Option(ExpressionEncoder[Short]()) :: Option(ExpressionEncoder[Double]()) :: Nil) val expected4 = If( IsNull(short), nullResult, @@ -362,8 +364,12 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("SPARK-24891 Fix HandleNullInputsForUDF rule") { val a = testRelation.output(0) val func = (x: Int, y: Int) => x + y - val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil, false :: false :: Nil) - val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil, false :: false :: Nil) + val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil, + Option(ExpressionEncoder[java.lang.Integer]()) :: + Option(ExpressionEncoder[java.lang.Integer]()) :: Nil) + val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil, + Option(ExpressionEncoder[java.lang.Integer]()) :: + Option(ExpressionEncoder[java.lang.Integer]()) :: Nil) val plan = Project(Alias(udf2, "")() :: Nil, testRelation) comparePlans(plan.analyze, plan.analyze.analyze) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 3f8d409992381..4473c20b2cca6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -45,6 +45,8 @@ trait AnalysisTest extends PlanTest { catalog.createTempView("TaBlE", TestRelations.testRelation, overrideIfExists = true) catalog.createTempView("TaBlE2", TestRelations.testRelation2, overrideIfExists = true) catalog.createTempView("TaBlE3", TestRelations.testRelation3, overrideIfExists = true) + catalog.createGlobalTempView("TaBlE4", TestRelations.testRelation4, overrideIfExists = true) + catalog.createGlobalTempView("TaBlE5", TestRelations.testRelation5, overrideIfExists = true) new Analyzer(catalog, conf) { override val extendedResolutionRules = EliminateSubqueryAliases +: extendedAnalysisRules } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index 5e66c038738a4..d3bd5d07a0932 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -241,4 +241,57 @@ class ResolveHintsSuite extends AnalysisTest { Project(testRelation.output, testRelation), caseSensitive = false) } + + test("Supports multi-part table names for join strategy hint resolution") { + Seq(("MAPJOIN", BROADCAST), + ("MERGEJOIN", SHUFFLE_MERGE), + ("SHUFFLE_HASH", SHUFFLE_HASH), + ("SHUFFLE_REPLICATE_NL", SHUFFLE_REPLICATE_NL)).foreach { case (hintName, st) => + // local temp table (single-part identifier case) + checkAnalysis( + UnresolvedHint(hintName, Seq("table", "table2"), + table("TaBlE").join(table("TaBlE2"))), + Join( + ResolvedHint(testRelation, HintInfo(strategy = Some(st))), + ResolvedHint(testRelation2, HintInfo(strategy = Some(st))), + Inner, + None, + JoinHint.NONE), + caseSensitive = false) + + checkAnalysis( + UnresolvedHint(hintName, Seq("TaBlE", "table2"), + table("TaBlE").join(table("TaBlE2"))), + Join( + ResolvedHint(testRelation, HintInfo(strategy = Some(st))), + testRelation2, + Inner, + None, + JoinHint.NONE), + caseSensitive = true) + + // global temp table (multi-part identifier case) + checkAnalysis( + UnresolvedHint(hintName, Seq("GlOBal_TeMP.table4", "table5"), + table("global_temp", "table4").join(table("global_temp", "table5"))), + Join( + ResolvedHint(testRelation4, HintInfo(strategy = Some(st))), + ResolvedHint(testRelation5, HintInfo(strategy = Some(st))), + Inner, + None, + JoinHint.NONE), + caseSensitive = false) + + checkAnalysis( + UnresolvedHint(hintName, Seq("global_temp.TaBlE4", "table5"), + table("global_temp", "TaBlE4").join(table("global_temp", "TaBlE5"))), + Join( + ResolvedHint(testRelation4, HintInfo(strategy = Some(st))), + testRelation5, + Inner, + None, + JoinHint.NONE), + caseSensitive = true) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TestRelations.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TestRelations.scala index e12e272aedffe..33b6029070938 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TestRelations.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TestRelations.scala @@ -44,6 +44,8 @@ object TestRelations { AttributeReference("g", StringType)(), AttributeReference("h", MapType(IntegerType, IntegerType))()) + val testRelation5 = LocalRelation(AttributeReference("i", StringType)()) + val nestedRelation = LocalRelation( AttributeReference("top", StructType( StructField("duplicateField", StringType) :: 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 0d6f9bcedb6a2..ab21a9ea5ba18 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 @@ -30,6 +30,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval class TypeCoercionSuite extends AnalysisTest { + import TypeCoercionSuite._ // scalastyle:off line.size.limit // The following table shows all implicit data type conversions that are not visible to the user. @@ -99,22 +100,6 @@ class TypeCoercionSuite extends AnalysisTest { case _ => Literal.create(null, dataType) } - val integralTypes: Seq[DataType] = - Seq(ByteType, ShortType, IntegerType, LongType) - val fractionalTypes: Seq[DataType] = - Seq(DoubleType, FloatType, DecimalType.SYSTEM_DEFAULT, DecimalType(10, 2)) - val numericTypes: Seq[DataType] = integralTypes ++ fractionalTypes - val atomicTypes: Seq[DataType] = - numericTypes ++ Seq(BinaryType, BooleanType, StringType, DateType, TimestampType) - val complexTypes: Seq[DataType] = - Seq(ArrayType(IntegerType), - ArrayType(StringType), - MapType(StringType, StringType), - new StructType().add("a1", StringType), - new StructType().add("a1", StringType).add("a2", IntegerType)) - val allTypes: Seq[DataType] = - atomicTypes ++ complexTypes ++ Seq(NullType, CalendarIntervalType) - // Check whether the type `checkedType` can be cast to all the types in `castableTypes`, // but cannot be cast to the other types in `allTypes`. private def checkTypeCasting(checkedType: DataType, castableTypes: Seq[DataType]): Unit = { @@ -467,11 +452,21 @@ class TypeCoercionSuite extends AnalysisTest { ArrayType(IntegerType, containsNull = false), Some(ArrayType(IntegerType, containsNull = true))) + widenTest( + ArrayType(NullType, containsNull = true), + ArrayType(IntegerType, containsNull = false), + Some(ArrayType(IntegerType, containsNull = true))) + widenTest( MapType(IntegerType, StringType, valueContainsNull = true), MapType(IntegerType, StringType, valueContainsNull = false), Some(MapType(IntegerType, StringType, valueContainsNull = true))) + widenTest( + MapType(NullType, NullType, true), + MapType(IntegerType, StringType, false), + Some(MapType(IntegerType, StringType, true))) + widenTest( new StructType() .add("arr", ArrayType(IntegerType, containsNull = true), nullable = false), @@ -479,6 +474,31 @@ class TypeCoercionSuite extends AnalysisTest { .add("arr", ArrayType(IntegerType, containsNull = false), nullable = true), Some(new StructType() .add("arr", ArrayType(IntegerType, containsNull = true), nullable = true))) + + widenTest( + new StructType() + .add("null", NullType, nullable = true), + new StructType() + .add("null", IntegerType, nullable = false), + Some(new StructType() + .add("null", IntegerType, nullable = true))) + + widenTest( + ArrayType(NullType, containsNull = false), + ArrayType(IntegerType, containsNull = false), + Some(ArrayType(IntegerType, containsNull = false))) + + widenTest(MapType(NullType, NullType, false), + MapType(IntegerType, StringType, false), + Some(MapType(IntegerType, StringType, false))) + + widenTest( + new StructType() + .add("null", NullType, nullable = false), + new StructType() + .add("null", IntegerType, nullable = false), + Some(new StructType() + .add("null", IntegerType, nullable = false))) } test("wider common type for decimal and array") { @@ -710,8 +730,6 @@ class TypeCoercionSuite extends AnalysisTest { } test("cast NullType for expressions that implement ExpectsInputTypes") { - import TypeCoercionSuite._ - ruleTest(TypeCoercion.ImplicitTypeCasts, AnyTypeUnaryExpression(Literal.create(null, NullType)), AnyTypeUnaryExpression(Literal.create(null, NullType))) @@ -722,8 +740,6 @@ class TypeCoercionSuite extends AnalysisTest { } test("cast NullType for binary operators") { - import TypeCoercionSuite._ - ruleTest(TypeCoercion.ImplicitTypeCasts, AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)), AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType))) @@ -1530,6 +1546,22 @@ class TypeCoercionSuite extends AnalysisTest { object TypeCoercionSuite { + val integralTypes: Seq[DataType] = + Seq(ByteType, ShortType, IntegerType, LongType) + val fractionalTypes: Seq[DataType] = + Seq(DoubleType, FloatType, DecimalType.SYSTEM_DEFAULT, DecimalType(10, 2)) + val numericTypes: Seq[DataType] = integralTypes ++ fractionalTypes + val atomicTypes: Seq[DataType] = + numericTypes ++ Seq(BinaryType, BooleanType, StringType, DateType, TimestampType) + val complexTypes: Seq[DataType] = + Seq(ArrayType(IntegerType), + ArrayType(StringType), + MapType(StringType, StringType), + new StructType().add("a1", StringType), + new StructType().add("a1", StringType).add("a2", IntegerType)) + val allTypes: Seq[DataType] = + atomicTypes ++ complexTypes ++ Seq(NullType, CalendarIntervalType) + case class AnyTypeUnaryExpression(child: Expression) extends UnaryExpression with ExpectsInputTypes with Unevaluable { override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala index ee73da33a1a90..b014eb92fae50 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types._ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { test("String fields types are inferred correctly from null types") { - val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "GMT") + val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "UTC") val inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(NullType, "") == NullType) @@ -48,7 +48,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("String fields types are inferred correctly from other types") { - val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "GMT") + val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "UTC") val inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(LongType, "1.0") == DoubleType) @@ -69,18 +69,18 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("Timestamp field types are inferred correctly via custom data format") { - var options = new CSVOptions(Map("timestampFormat" -> "yyyy-mm"), false, "GMT") + var options = new CSVOptions(Map("timestampFormat" -> "yyyy-mm"), false, "UTC") var inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(TimestampType, "2015-08") == TimestampType) - options = new CSVOptions(Map("timestampFormat" -> "yyyy"), false, "GMT") + options = new CSVOptions(Map("timestampFormat" -> "yyyy"), false, "UTC") inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(TimestampType, "2015") == TimestampType) } test("Timestamp field types are inferred correctly from other types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") val inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(IntegerType, "2015-08-20 14") == StringType) @@ -89,7 +89,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("Boolean fields types are inferred correctly from other types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") val inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(LongType, "Fale") == StringType) @@ -97,7 +97,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("Type arrays are merged to highest common type") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") val inferSchema = new CSVInferSchema(options) assert( @@ -112,14 +112,14 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("Null fields are handled properly when a nullValue is specified") { - var options = new CSVOptions(Map("nullValue" -> "null"), false, "GMT") + var options = new CSVOptions(Map("nullValue" -> "null"), false, "UTC") var inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(NullType, "null") == NullType) assert(inferSchema.inferField(StringType, "null") == StringType) assert(inferSchema.inferField(LongType, "null") == LongType) - options = new CSVOptions(Map("nullValue" -> "\\N"), false, "GMT") + options = new CSVOptions(Map("nullValue" -> "\\N"), false, "UTC") inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(IntegerType, "\\N") == IntegerType) @@ -130,7 +130,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("Merging Nulltypes should yield Nulltype.") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") val inferSchema = new CSVInferSchema(options) val mergedNullTypes = inferSchema.mergeRowTypes(Array(NullType), Array(NullType)) @@ -138,14 +138,14 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { - val options = new CSVOptions(Map("TiMeStampFormat" -> "yyyy-mm"), false, "GMT") + val options = new CSVOptions(Map("TiMeStampFormat" -> "yyyy-mm"), false, "UTC") val inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(TimestampType, "2015-08") == TimestampType) } test("SPARK-18877: `inferField` on DecimalType should find a common type with `typeSoFar`") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") val inferSchema = new CSVInferSchema(options) withSQLConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key -> "true") { @@ -166,7 +166,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { test("DoubleType should be inferred when user defined nan/inf are provided") { val options = new CSVOptions(Map("nanValue" -> "nan", "negativeInf" -> "-inf", - "positiveInf" -> "inf"), false, "GMT") + "positiveInf" -> "inf"), false, "UTC") val inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(NullType, "nan") == DoubleType) @@ -179,7 +179,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { val options = new CSVOptions( parameters = Map("locale" -> langTag, "inferSchema" -> "true", "sep" -> "|"), columnPruning = false, - defaultTimeZoneId = "GMT") + defaultTimeZoneId = "UTC") val inferSchema = new CSVInferSchema(options) val df = new DecimalFormat("", new DecimalFormatSymbols(Locale.forLanguageTag(langTag))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index b605e9944e4a0..0e9fcc980aabb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.csv import java.math.BigDecimal import java.text.{DecimalFormat, DecimalFormatSymbols} -import java.time.ZoneOffset import java.util.{Locale, TimeZone} import org.apache.commons.lang3.time.FastDateFormat @@ -44,7 +43,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { stringValues.zip(decimalValues).foreach { case (strVal, decimalVal) => val decimalValue = new BigDecimal(decimalVal.toString) - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") val parser = new UnivocityParser(StructType(Seq.empty), options) assert(parser.makeConverter("_1", decimalType).apply(strVal) === Decimal(decimalValue, decimalType.precision, decimalType.scale)) @@ -58,14 +57,14 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { // Nullable field with nullValue option. types.foreach { t => // Tests that a custom nullValue. - val nullValueOptions = new CSVOptions(Map("nullValue" -> "-"), false, "GMT") + val nullValueOptions = new CSVOptions(Map("nullValue" -> "-"), false, "UTC") var parser = new UnivocityParser(StructType(Seq.empty), nullValueOptions) val converter = parser.makeConverter("_1", t, nullable = true) assertNull(converter.apply("-")) assertNull(converter.apply(null)) // Tests that the default nullValue is empty string. - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") parser = new UnivocityParser(StructType(Seq.empty), options) assertNull(parser.makeConverter("_1", t, nullable = true).apply("")) } @@ -73,7 +72,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { // Not nullable field with nullValue option. types.foreach { t => // Casts a null to not nullable field should throw an exception. - val options = new CSVOptions(Map("nullValue" -> "-"), false, "GMT") + val options = new CSVOptions(Map("nullValue" -> "-"), false, "UTC") val parser = new UnivocityParser(StructType(Seq.empty), options) val converter = parser.makeConverter("_1", t, nullable = false) var message = intercept[RuntimeException] { @@ -89,7 +88,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { // If nullValue is different with empty string, then, empty string should not be casted into // null. Seq(true, false).foreach { b => - val options = new CSVOptions(Map("nullValue" -> "null"), false, "GMT") + val options = new CSVOptions(Map("nullValue" -> "null"), false, "UTC") val parser = new UnivocityParser(StructType(Seq.empty), options) val converter = parser.makeConverter("_1", StringType, nullable = b) assert(converter.apply("") == UTF8String.fromString("")) @@ -97,7 +96,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { } test("Throws exception for empty string with non null type") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") val parser = new UnivocityParser(StructType(Seq.empty), options) val exception = intercept[RuntimeException]{ parser.makeConverter("_1", IntegerType, nullable = false).apply("") @@ -106,7 +105,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { } test("Types are cast correctly") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") var parser = new UnivocityParser(StructType(Seq.empty), options) assert(parser.makeConverter("_1", ByteType).apply("10") == 10) assert(parser.makeConverter("_1", ShortType).apply("10") == 10) @@ -117,7 +116,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { assert(parser.makeConverter("_1", BooleanType).apply("true") == true) var timestampsOptions = - new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy HH:mm"), false, "GMT") + new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy HH:mm"), false, "UTC") parser = new UnivocityParser(StructType(Seq.empty), timestampsOptions) val customTimestamp = "31/01/2015 00:00" var format = FastDateFormat.getInstance( @@ -130,7 +129,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { assert(castedTimestamp == expectedTime * 1000L) val customDate = "31/01/2015" - val dateOptions = new CSVOptions(Map("dateFormat" -> "dd/MM/yyyy"), false, "GMT") + val dateOptions = new CSVOptions(Map("dateFormat" -> "dd/MM/yyyy"), false, "UTC") parser = new UnivocityParser(StructType(Seq.empty), dateOptions) format = FastDateFormat.getInstance( dateOptions.dateFormat, @@ -139,7 +138,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { val expectedDate = DateTimeUtils.millisToMicros(format.parse(customDate).getTime) val castedDate = parser.makeConverter("_1", DateType, nullable = true) .apply(customDate) - assert(castedDate == DateTimeUtils.microsToDays(expectedDate, ZoneOffset.UTC)) + assert(castedDate == DateTimeUtils.microsToDays(expectedDate, UTC)) val timestamp = "2015-01-01 00:00:00" timestampsOptions = new CSVOptions(Map( @@ -154,7 +153,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { } test("Throws exception for casting an invalid string to Float and Double Types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") val parser = new UnivocityParser(StructType(Seq.empty), options) val types = Seq(DoubleType, FloatType) val input = Seq("10u000", "abc", "1 2/3") @@ -169,7 +168,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { } test("Float NaN values are parsed correctly") { - val options = new CSVOptions(Map("nanValue" -> "nn"), false, "GMT") + val options = new CSVOptions(Map("nanValue" -> "nn"), false, "UTC") val parser = new UnivocityParser(StructType(Seq.empty), options) val floatVal: Float = parser.makeConverter( "_1", FloatType, nullable = true).apply("nn").asInstanceOf[Float] @@ -180,7 +179,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { } test("Double NaN values are parsed correctly") { - val options = new CSVOptions(Map("nanValue" -> "-"), false, "GMT") + val options = new CSVOptions(Map("nanValue" -> "-"), false, "UTC") val parser = new UnivocityParser(StructType(Seq.empty), options) val doubleVal: Double = parser.makeConverter( "_1", DoubleType, nullable = true).apply("-").asInstanceOf[Double] @@ -189,14 +188,14 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { } test("Float infinite values can be parsed") { - val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT") + val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "UTC") var parser = new UnivocityParser(StructType(Seq.empty), negativeInfOptions) val floatVal1 = parser.makeConverter( "_1", FloatType, nullable = true).apply("max").asInstanceOf[Float] assert(floatVal1 == Float.NegativeInfinity) - val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "GMT") + val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "UTC") parser = new UnivocityParser(StructType(Seq.empty), positiveInfOptions) val floatVal2 = parser.makeConverter( "_1", FloatType, nullable = true).apply("max").asInstanceOf[Float] @@ -205,14 +204,14 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { } test("Double infinite values can be parsed") { - val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT") + val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "UTC") var parser = new UnivocityParser(StructType(Seq.empty), negativeInfOptions) val doubleVal1 = parser.makeConverter( "_1", DoubleType, nullable = true).apply("max").asInstanceOf[Double] assert(doubleVal1 == Double.NegativeInfinity) - val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "GMT") + val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "UTC") parser = new UnivocityParser(StructType(Seq.empty), positiveInfOptions) val doubleVal2 = parser.makeConverter( "_1", DoubleType, nullable = true).apply("max").asInstanceOf[Double] @@ -228,7 +227,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { val df = new DecimalFormat("", new DecimalFormatSymbols(Locale.forLanguageTag(langTag))) val input = df.format(expected.toBigDecimal) - val options = new CSVOptions(Map("locale" -> langTag), false, "GMT") + val options = new CSVOptions(Map("locale" -> langTag), false, "UTC") val parser = new UnivocityParser(new StructType().add("d", decimalType), options) assert(parser.makeConverter("_1", decimalType).apply(input) === expected) @@ -263,7 +262,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { val input = "name\t42" val expected = UTF8String.fromString(input) - val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val options = new CSVOptions(Map.empty[String, String], false, "UTC") val parser = new UnivocityParser(StructType(Seq.empty), options) val convertedValue = parser.makeConverter("_1", StringBasedUDT, nullable = false).apply(input) @@ -280,7 +279,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { filters: Seq[Filter], expected: Option[InternalRow]): Unit = { Seq(false, true).foreach { columnPruning => - val options = new CSVOptions(Map.empty[String, String], columnPruning, "GMT") + val options = new CSVOptions(Map.empty[String, String], columnPruning, "UTC") val parser = new UnivocityParser(dataSchema, requiredSchema, options, filters) val actual = parser.parse(input) assert(actual === expected) @@ -355,8 +354,8 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { val options = new CSVOptions(Map.empty[String, String], false, "UTC") check(new UnivocityParser(StructType(Seq.empty), options)) - val optionsWithPattern = - new CSVOptions(Map("timestampFormat" -> "invalid", "dateFormat" -> "invalid"), false, "UTC") + val optionsWithPattern = new CSVOptions( + Map("timestampFormat" -> "invalid", "dateFormat" -> "invalid"), false, "UTC") check(new UnivocityParser(StructType(Seq.empty), optionsWithPattern)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index c1f1be3b30e4b..1036dc725c205 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.util.ArrayData import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.ClosureCleaner case class RepeatedStruct(s: Seq[PrimitiveData]) @@ -107,6 +107,8 @@ class UDTForCaseClass extends UserDefinedType[UDTCaseClass] { } } +case class Bar(i: Any) +case class Foo(i: Bar) extends AnyVal case class PrimitiveValueClass(wrapped: Int) extends AnyVal case class ReferenceValueClass(wrapped: ReferenceValueClass.Container) extends AnyVal object ReferenceValueClass { @@ -207,9 +209,9 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes productTest( OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true), - Some(PrimitiveData(1, 1, 1, 1, 1, 1, true)))) + Some(PrimitiveData(1, 1, 1, 1, 1, 1, true)), Some(new CalendarInterval(1, 2, 3)))) - productTest(OptionalData(None, None, None, None, None, None, None, None)) + productTest(OptionalData(None, None, None, None, None, None, None, None, None)) encodeDecodeTest(Seq(Some(1), None), "Option in array") encodeDecodeTest(Map(1 -> Some(10L), 2 -> Some(20L), 3 -> None), "Option in map") @@ -311,6 +313,13 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes productTest(("UDT", new ExamplePoint(0.1, 0.2))) + test("AnyVal class with Any fields") { + val exception = intercept[UnsupportedOperationException](implicitly[ExpressionEncoder[Foo]]) + val errorMsg = exception.getMessage + assert(errorMsg.contains("root class: \"org.apache.spark.sql.catalyst.encoders.Foo\"")) + assert(errorMsg.contains("No Encoder found for Any")) + } + test("nullable of encoder schema") { def checkNullable[T: ExpressionEncoder](nullable: Boolean*): Unit = { assert(implicitly[ExpressionEncoder[T]].schema.map(_.nullable) === nullable.toSeq) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala index 88d4d460751b6..d8f3ad24246a3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala @@ -21,6 +21,7 @@ import java.sql.Timestamp import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.types.{IntegerType, StringType} /** A static class for testing purpose. */ @@ -101,6 +102,11 @@ class CallMethodViaReflectionSuite extends SparkFunSuite with ExpressionEvalHelp checkEvaluation(createExpr(staticClassName, "method4", 4, "four"), "m4four") } + test("escaping of class and method names") { + GenerateUnsafeProjection.generate( + CallMethodViaReflection(Seq(Literal("\"quote"), Literal("\"quote"), Literal(null))) :: Nil) + } + private def createExpr(className: String, methodName: String, args: Any*) = { CallMethodViaReflection( Literal.create(className, StringType) +: 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 302a246c39377..ee94f3587b55c 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 @@ -26,6 +26,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCoercion.numericPrecedence +import org.apache.spark.sql.catalyst.analysis.TypeCoercionSuite import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, CollectSet} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.util.DateTimeConstants._ @@ -49,7 +50,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } protected def checkNullCast(from: DataType, to: DataType): Unit = { - checkEvaluation(cast(Literal.create(null, from), to, Option("GMT")), null) + checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) } test("null cast") { @@ -111,13 +112,14 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("cast string to timestamp") { - new ParVector(ALL_TIMEZONES.toVector).foreach { tz => + new ParVector(ALL_TIMEZONES.toVector).foreach { zid => def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { - checkEvaluation(cast(Literal(str), TimestampType, Option(tz.getID)), expected) + checkEvaluation(cast(Literal(str), TimestampType, Option(zid.getId)), expected) } checkCastStringToTimestamp("123", null) + val tz = TimeZone.getTimeZone(zid) var c = Calendar.getInstance(tz) c.set(2015, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) @@ -141,7 +143,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { // 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 = 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)) @@ -172,7 +174,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { // 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 = 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)) @@ -262,10 +264,10 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { val zts = sd + " 00:00:00" val sts = sd + " 00:00:02" val nts = sts + ".1" - val ts = withDefaultTimeZone(TimeZoneGMT)(Timestamp.valueOf(nts)) + val ts = withDefaultTimeZone(UTC)(Timestamp.valueOf(nts)) for (tz <- ALL_TIMEZONES) { - val timeZoneId = Option(tz.getID) + val timeZoneId = Option(tz.getId) var c = Calendar.getInstance(TimeZoneGMT) c.set(2015, 2, 8, 2, 30, 0) checkEvaluation( @@ -280,23 +282,23 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { millisToMicros(c.getTimeInMillis)) } - val gmtId = Option("GMT") - checkEvaluation(cast("abdef", StringType), "abdef") - checkEvaluation(cast("abdef", TimestampType, gmtId), null) + checkEvaluation(cast("abdef", TimestampType, UTC_OPT), 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, gmtId), StringType, gmtId), nts) + checkEvaluation(cast(cast(nts, TimestampType, UTC_OPT), StringType, UTC_OPT), nts) checkEvaluation( - cast(cast(ts, StringType, gmtId), TimestampType, gmtId), + cast(cast(ts, StringType, UTC_OPT), TimestampType, UTC_OPT), DateTimeUtils.fromJavaTimestamp(ts)) // all convert to string type to check - checkEvaluation(cast(cast(cast(nts, TimestampType, gmtId), DateType, gmtId), StringType), sd) checkEvaluation( - cast(cast(cast(ts, DateType, gmtId), TimestampType, gmtId), StringType, gmtId), + cast(cast(cast(nts, TimestampType, UTC_OPT), DateType, UTC_OPT), StringType), + sd) + checkEvaluation( + cast(cast(cast(ts, DateType, UTC_OPT), TimestampType, UTC_OPT), StringType, UTC_OPT), zts) checkEvaluation(cast(cast("abdef", BinaryType), StringType), "abdef") @@ -309,7 +311,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), 5.toShort) checkEvaluation( - cast(cast(cast(cast(cast(cast("5", TimestampType, gmtId), ByteType), + cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), null) checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), @@ -360,8 +362,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(d, DecimalType(10, 2)), null) checkEvaluation(cast(d, StringType), "1970-01-01") - val gmtId = Option("GMT") - checkEvaluation(cast(cast(d, TimestampType, gmtId), StringType, gmtId), "1970-01-01 00:00:00") + checkEvaluation( + cast(cast(d, TimestampType, UTC_OPT), StringType, UTC_OPT), + "1970-01-01 00:00:00") } test("cast from timestamp") { @@ -412,6 +415,14 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { assert(ret.resolved) checkEvaluation(ret, Seq(null, true, false, null)) } + + { + val array = Literal.create(Seq.empty, ArrayType(NullType, containsNull = false)) + val ret = cast(array, ArrayType(IntegerType, containsNull = false)) + assert(ret.resolved) + checkEvaluation(ret, Seq.empty) + } + { val ret = cast(array, ArrayType(BooleanType, containsNull = false)) assert(ret.resolved === false) @@ -1140,6 +1151,36 @@ class CastSuite extends CastSuiteBase { assert(Cast.canCast(set.dataType, ArrayType(StringType, false))) } + test("NullTypes should be able to cast to any complex types") { + assert(Cast.canCast(ArrayType(NullType, true), ArrayType(IntegerType, true))) + assert(Cast.canCast(ArrayType(NullType, false), ArrayType(IntegerType, true))) + + assert(Cast.canCast( + MapType(NullType, NullType, true), MapType(IntegerType, IntegerType, true))) + assert(Cast.canCast( + MapType(NullType, NullType, false), MapType(IntegerType, IntegerType, true))) + + assert(Cast.canCast( + StructType(StructField("a", NullType, true) :: Nil), + StructType(StructField("a", IntegerType, true) :: Nil))) + assert(Cast.canCast( + StructType(StructField("a", NullType, false) :: Nil), + StructType(StructField("a", IntegerType, true) :: Nil))) + } + + test("SPARK-31227: Non-nullable null type should not coerce to nullable type") { + TypeCoercionSuite.allTypes.foreach { t => + assert(Cast.canCast(ArrayType(NullType, false), ArrayType(t, false))) + + assert(Cast.canCast( + MapType(NullType, NullType, false), MapType(t, t, false))) + + assert(Cast.canCast( + StructType(StructField("a", NullType, false) :: Nil), + StructType(StructField("a", t, false) :: Nil))) + } + } + test("Cast should output null for invalid strings when ANSI is not enabled.") { withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(cast("abdef", DecimalType.USER_DEFAULT), null) @@ -1287,6 +1328,8 @@ class AnsiCastSuite extends CastSuiteBase { cast("123-string", dataType), "invalid input") checkExceptionInExpression[NumberFormatException]( cast("2020-07-19", dataType), "invalid input") + checkExceptionInExpression[NumberFormatException]( + cast("1.23", dataType), "invalid input") } Seq(DoubleType, FloatType, DecimalType.USER_DEFAULT).foreach { dataType => 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 b7628a373d0bd..db52229f860b4 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 @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.LA import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.ThreadUtils @@ -190,7 +191,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val expressions = Seq.fill(length) { ToUTCTimestamp( Literal.create(Timestamp.valueOf("2015-07-24 00:00:00"), TimestampType), - Literal.create("PST", StringType)) + Literal.create(LA.getId, StringType)) } val plan = GenerateMutableProjection.generate(expressions) val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) @@ -207,7 +208,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val expressions = Seq.fill(length) { ToUTCTimestamp( Literal.create(Timestamp.valueOf("2017-10-10 00:00:00"), TimestampType), - Literal.create("PST", StringType)) + Literal.create(LA.getId, StringType)) } val plan = GenerateMutableProjection.generate(expressions) val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 3cfc66f5cdb03..a4baad0c72444 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -27,8 +27,9 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY -import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -74,6 +75,12 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper withSQLConf(SQLConf.LEGACY_SIZE_OF_NULL.key -> "false") { testSize(sizeOfNull = null) } + // size(null) should return null under ansi mode. + withSQLConf( + SQLConf.LEGACY_SIZE_OF_NULL.key -> "true", + SQLConf.ANSI_ENABLED.key -> "true") { + testSize(sizeOfNull = null) + } } test("MapKeys/MapValues") { @@ -839,20 +846,22 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper test("Sequence on DST boundaries") { val timeZone = TimeZone.getTimeZone("Europe/Prague") - val dstOffset = timeZone.getDSTSavings - def noDST(t: Timestamp): Timestamp = new Timestamp(t.getTime - dstOffset) + def ts(s: String, noDST: Boolean = false): Long = { + val offset = if (noDST) timeZone.getDSTSavings else 0 + DateTimeUtils.millisToMicros(Timestamp.valueOf(s).getTime - offset) + } - DateTimeTestUtils.withDefaultTimeZone(timeZone) { + DateTimeTestUtils.withDefaultTimeZone(timeZone.toZoneId) { // Spring time change checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-25 01:30:00")), Literal(Timestamp.valueOf("2018-03-25 03:30:00")), Literal(stringToInterval("interval 30 minutes"))), Seq( - Timestamp.valueOf("2018-03-25 01:30:00"), - Timestamp.valueOf("2018-03-25 03:00:00"), - Timestamp.valueOf("2018-03-25 03:30:00"))) + ts("2018-03-25 01:30:00"), + ts("2018-03-25 03:00:00"), + ts("2018-03-25 03:30:00"))) // Autumn time change checkEvaluation(new Sequence( @@ -860,18 +869,18 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper Literal(Timestamp.valueOf("2018-10-28 03:30:00")), Literal(stringToInterval("interval 30 minutes"))), Seq( - Timestamp.valueOf("2018-10-28 01:30:00"), - noDST(Timestamp.valueOf("2018-10-28 02:00:00")), - noDST(Timestamp.valueOf("2018-10-28 02:30:00")), - Timestamp.valueOf("2018-10-28 02:00:00"), - Timestamp.valueOf("2018-10-28 02:30:00"), - Timestamp.valueOf("2018-10-28 03:00:00"), - Timestamp.valueOf("2018-10-28 03:30:00"))) + ts("2018-10-28 01:30:00"), + ts("2018-10-28 02:00:00", noDST = true), + ts("2018-10-28 02:30:00", noDST = true), + ts("2018-10-28 02:00:00"), + ts("2018-10-28 02:30:00"), + ts("2018-10-28 03:00:00"), + ts("2018-10-28 03:30:00"))) } } test("Sequence of dates") { - DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone("UTC")) { + DateTimeTestUtils.withDefaultTimeZone(UTC) { checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-01-05")), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala index e623910e2efe1..16d78b0526503 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.text.SimpleDateFormat -import java.util.{Calendar, Locale} +import java.util.{Calendar, Locale, TimeZone} import org.scalatest.exceptions.TestFailedException @@ -27,19 +27,18 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{PST, UTC_OPT} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with PlanTestBase { val badCsv = "\u0000\u0000\u0000A\u0001AAA" - val gmtId = Option(DateTimeUtils.TimeZoneGMT.getID) - test("from_csv") { val csvData = "1" val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( - CsvToStructs(schema, Map.empty, Literal(csvData), gmtId), + CsvToStructs(schema, Map.empty, Literal(csvData), UTC_OPT), InternalRow(1) ) } @@ -48,17 +47,17 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P val csvData = "---" val schema = StructType(StructField("a", DoubleType) :: Nil) checkEvaluation( - CsvToStructs(schema, Map("mode" -> PermissiveMode.name), Literal(csvData), gmtId), + CsvToStructs(schema, Map("mode" -> PermissiveMode.name), Literal(csvData), UTC_OPT), InternalRow(null)) // Default mode is Permissive - checkEvaluation(CsvToStructs(schema, Map.empty, Literal(csvData), gmtId), InternalRow(null)) + checkEvaluation(CsvToStructs(schema, Map.empty, Literal(csvData), UTC_OPT), InternalRow(null)) } test("from_csv null input column") { val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( - CsvToStructs(schema, Map.empty, Literal.create(null, StringType), gmtId), + CsvToStructs(schema, Map.empty, Literal.create(null, StringType), UTC_OPT), null ) } @@ -66,7 +65,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P test("from_csv bad UTF-8") { val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( - CsvToStructs(schema, Map.empty, Literal(badCsv), gmtId), + CsvToStructs(schema, Map.empty, Literal(badCsv), UTC_OPT), InternalRow(null)) } @@ -78,20 +77,20 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P c.set(2016, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 123) checkEvaluation( - CsvToStructs(schema, Map.empty, Literal(csvData1), gmtId), + CsvToStructs(schema, Map.empty, Literal(csvData1), UTC_OPT), InternalRow(c.getTimeInMillis * 1000L) ) // The result doesn't change because the CSV string includes timezone string ("Z" here), // which means the string represents the timestamp string in the timezone regardless of // the timeZoneId parameter. checkEvaluation( - CsvToStructs(schema, Map.empty, Literal(csvData1), Option("PST")), + CsvToStructs(schema, Map.empty, Literal(csvData1), Option(PST.getId)), InternalRow(c.getTimeInMillis * 1000L) ) val csvData2 = "2016-01-01T00:00:00" - for (tz <- DateTimeTestUtils.outstandingTimezones) { - c = Calendar.getInstance(tz) + for (zid <- DateTimeTestUtils.outstandingZoneIds) { + c = Calendar.getInstance(TimeZone.getTimeZone(zid)) c.set(2016, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) checkEvaluation( @@ -99,16 +98,16 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P schema, Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss"), Literal(csvData2), - Option(tz.getID)), + Option(zid.getId)), InternalRow(c.getTimeInMillis * 1000L) ) checkEvaluation( CsvToStructs( schema, Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", - DateTimeUtils.TIMEZONE_OPTION -> tz.getID), + DateTimeUtils.TIMEZONE_OPTION -> zid.getId), Literal(csvData2), - gmtId), + UTC_OPT), InternalRow(c.getTimeInMillis * 1000L) ) } @@ -117,7 +116,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P test("from_csv empty input column") { val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( - CsvToStructs(schema, Map.empty, Literal.create(" ", StringType), gmtId), + CsvToStructs(schema, Map.empty, Literal.create(" ", StringType), UTC_OPT), InternalRow(null) ) } @@ -129,7 +128,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P .add("b", StringType, nullable = false) .add("c", StringType, nullable = false) val output = InternalRow(1L, null, UTF8String.fromString("foo")) - val expr = CsvToStructs(csvSchema, Map.empty, Literal.create(input, StringType), gmtId) + val expr = CsvToStructs(csvSchema, Map.empty, Literal.create(input, StringType), UTC_OPT) checkEvaluation(expr, output) val schema = expr.dataType val schemaToCompare = csvSchema.asNullable @@ -142,7 +141,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P .add("a", IntegerType) .add("b", IntegerType) checkEvaluation( - CsvToStructs(schema, Map.empty, Literal.create("1"), gmtId), + CsvToStructs(schema, Map.empty, Literal.create("1"), UTC_OPT), InternalRow(1, null) ) } @@ -152,7 +151,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P val schema = StructType(StructField("a", DoubleType) :: Nil) val exception = intercept[TestFailedException] { checkEvaluation( - CsvToStructs(schema, Map("mode" -> DropMalformedMode.name), Literal(csvData), gmtId), + CsvToStructs(schema, Map("mode" -> DropMalformedMode.name), Literal(csvData), UTC_OPT), InternalRow(null)) }.getCause assert(exception.getMessage.contains("from_csv() doesn't support the DROPMALFORMED mode")) @@ -171,14 +170,14 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P test("to_csv - struct") { val schema = StructType(StructField("a", IntegerType) :: Nil) val struct = Literal.create(create_row(1), schema) - checkEvaluation(StructsToCsv(Map.empty, struct, gmtId), "1") + checkEvaluation(StructsToCsv(Map.empty, struct, UTC_OPT), "1") } test("to_csv null input column") { val schema = StructType(StructField("a", IntegerType) :: Nil) val struct = Literal.create(null, schema) checkEvaluation( - StructsToCsv(Map.empty, struct, gmtId), + StructsToCsv(Map.empty, struct, UTC_OPT), null ) } @@ -190,24 +189,24 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P c.set(Calendar.MILLISECOND, 0) val struct = Literal.create(create_row(c.getTimeInMillis * 1000L), schema) - checkEvaluation(StructsToCsv(Map.empty, struct, gmtId), "2016-01-01T00:00:00.000Z") + checkEvaluation(StructsToCsv(Map.empty, struct, UTC_OPT), "2016-01-01T00:00:00.000Z") checkEvaluation( - StructsToCsv(Map.empty, struct, Option("PST")), "2015-12-31T16:00:00.000-08:00") + StructsToCsv(Map.empty, struct, Option(PST.getId)), "2015-12-31T16:00:00.000-08:00") checkEvaluation( StructsToCsv( Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", - DateTimeUtils.TIMEZONE_OPTION -> gmtId.get), + DateTimeUtils.TIMEZONE_OPTION -> UTC_OPT.get), struct, - gmtId), + UTC_OPT), "2016-01-01T00:00:00" ) checkEvaluation( StructsToCsv( Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", - DateTimeUtils.TIMEZONE_OPTION -> "PST"), + DateTimeUtils.TIMEZONE_OPTION -> PST.getId), struct, - gmtId), + UTC_OPT), "2015-12-31T16:00:00" ) } @@ -223,7 +222,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P val options = Map("dateFormat" -> dateFormat, "locale" -> langTag) checkEvaluation( - CsvToStructs(schema, options, Literal.create(dateStr), gmtId), + CsvToStructs(schema, options, Literal.create(dateStr), UTC_OPT), InternalRow(17836)) // number of days from 1970-01-01 } } @@ -234,7 +233,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P schema = StructType.fromDDL("i int, _unparsed boolean"), options = Map("columnNameOfCorruptRecord" -> "_unparsed"), child = Literal.create("a"), - timeZoneId = gmtId), + timeZoneId = UTC_OPT), expectedErrMsg = "The field for corrupt records must be string type and nullable") } 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 7fced04fd5881..d9b508a563a8b 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,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{Instant, LocalDate, LocalDateTime, ZoneId, ZoneOffset} +import java.time.{Instant, LocalDate, LocalDateTime, ZoneId} import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit._ @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -38,15 +37,11 @@ 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) + private val PST_OPT = Option(PST.getId) + private val JST_OPT = Option(JST.getId) def toMillis(timestamp: String): Long = { - val tf = TimestampFormatter("yyyy-MM-dd HH:mm:ss", ZoneOffset.UTC) + val tf = TimestampFormatter("yyyy-MM-dd HH:mm:ss", UTC) DateTimeUtils.microsToMillis(tf.parse(timestamp)) } val date = "2015-04-08 13:10:15" @@ -55,13 +50,13 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val ts = new Timestamp(toMillis(time)) test("datetime function current_date") { - val d0 = DateTimeUtils.currentDate(ZoneOffset.UTC) - val cd = CurrentDate(gmtId).eval(EmptyRow).asInstanceOf[Int] - val d1 = DateTimeUtils.currentDate(ZoneOffset.UTC) + val d0 = DateTimeUtils.currentDate(UTC) + val cd = CurrentDate(UTC_OPT).eval(EmptyRow).asInstanceOf[Int] + val d1 = DateTimeUtils.currentDate(UTC) assert(d0 <= cd && cd <= d1 && d1 - d0 <= 1) - val cdjst = CurrentDate(jstId).eval(EmptyRow).asInstanceOf[Int] - val cdpst = CurrentDate(pstId).eval(EmptyRow).asInstanceOf[Int] + val cdjst = CurrentDate(JST_OPT).eval(EmptyRow).asInstanceOf[Int] + val cdpst = CurrentDate(PST_OPT).eval(EmptyRow).asInstanceOf[Int] assert(cdpst <= cd && cd <= cdjst) } @@ -93,8 +88,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(date), DateType, gmtId)), 2015) - checkEvaluation(Year(Cast(Literal(ts), DateType, gmtId)), 2013) + checkEvaluation(Year(Cast(Literal(date), DateType, UTC_OPT)), 2015) + checkEvaluation(Year(Cast(Literal(ts), DateType, UTC_OPT)), 2013) val c = Calendar.getInstance() (2000 to 2002).foreach { y => @@ -115,8 +110,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(date), DateType, gmtId)), 2) - checkEvaluation(Quarter(Cast(Literal(ts), DateType, gmtId)), 4) + checkEvaluation(Quarter(Cast(Literal(date), DateType, UTC_OPT)), 2) + checkEvaluation(Quarter(Cast(Literal(ts), DateType, UTC_OPT)), 4) val c = Calendar.getInstance() (2003 to 2004).foreach { y => @@ -138,8 +133,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("Month") { checkEvaluation(Month(Literal.create(null, DateType)), null) checkEvaluation(Month(Literal(d)), 4) - checkEvaluation(Month(Cast(Literal(date), DateType, gmtId)), 4) - checkEvaluation(Month(Cast(Literal(ts), DateType, gmtId)), 11) + checkEvaluation(Month(Cast(Literal(date), DateType, UTC_OPT)), 4) + checkEvaluation(Month(Cast(Literal(ts), DateType, UTC_OPT)), 11) checkEvaluation(Month(Cast(Literal("1582-04-28 13:10:15"), DateType)), 4) checkEvaluation(Month(Cast(Literal("1582-10-04 13:10:15"), DateType)), 10) @@ -163,8 +158,8 @@ 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(date), DateType, gmtId)), 8) - checkEvaluation(DayOfMonth(Cast(Literal(ts), DateType, gmtId)), 8) + checkEvaluation(DayOfMonth(Cast(Literal(date), DateType, UTC_OPT)), 8) + checkEvaluation(DayOfMonth(Cast(Literal(ts), DateType, UTC_OPT)), 8) checkEvaluation(DayOfMonth(Cast(Literal("1582-04-28 13:10:15"), DateType)), 28) checkEvaluation(DayOfMonth(Cast(Literal("1582-10-15 13:10:15"), DateType)), 15) @@ -183,16 +178,16 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Seconds") { - assert(Second(Literal.create(null, DateType), gmtId).resolved === false) - assert(Second(Cast(Literal(d), TimestampType, gmtId), gmtId).resolved ) - checkEvaluation(Second(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) - checkEvaluation(Second(Cast(Literal(date), TimestampType, gmtId), gmtId), 15) - checkEvaluation(Second(Literal(ts), gmtId), 15) + assert(Second(Literal.create(null, DateType), UTC_OPT).resolved === false) + assert(Second(Cast(Literal(d), TimestampType, UTC_OPT), UTC_OPT).resolved ) + checkEvaluation(Second(Cast(Literal(d), TimestampType, UTC_OPT), UTC_OPT), 0) + checkEvaluation(Second(Cast(Literal(date), TimestampType, UTC_OPT), UTC_OPT), 15) + checkEvaluation(Second(Literal(ts), UTC_OPT), 15) val c = Calendar.getInstance() - for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { - val timeZoneId = Option(tz.getID) - c.setTimeZone(tz) + for (zid <- outstandingZoneIds) { + val timeZoneId = Option(zid.getId) + c.setTimeZone(TimeZone.getTimeZone(zid)) (0 to 60 by 5).foreach { s => c.set(2015, 18, 3, 3, 5, s) checkEvaluation( @@ -207,10 +202,10 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("DayOfWeek") { checkEvaluation(DayOfWeek(Literal.create(null, DateType)), null) checkEvaluation(DayOfWeek(Literal(d)), Calendar.WEDNESDAY) - checkEvaluation(DayOfWeek(Cast(Literal(date), DateType, gmtId)), + checkEvaluation(DayOfWeek(Cast(Literal(date), DateType, UTC_OPT)), Calendar.WEDNESDAY) - checkEvaluation(DayOfWeek(Cast(Literal(ts), DateType, gmtId)), Calendar.FRIDAY) - checkEvaluation(DayOfWeek(Cast(Literal("2011-05-06"), DateType, gmtId)), Calendar.FRIDAY) + checkEvaluation(DayOfWeek(Cast(Literal(ts), DateType, UTC_OPT)), Calendar.FRIDAY) + checkEvaluation(DayOfWeek(Cast(Literal("2011-05-06"), DateType, UTC_OPT)), Calendar.FRIDAY) checkEvaluation(DayOfWeek(Literal(new Date(toMillis("2017-05-27 13:10:15")))), Calendar.SATURDAY) checkEvaluation(DayOfWeek(Literal(new Date(toMillis("1582-10-15 13:10:15")))), @@ -221,9 +216,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("WeekDay") { checkEvaluation(WeekDay(Literal.create(null, DateType)), null) checkEvaluation(WeekDay(Literal(d)), 2) - checkEvaluation(WeekDay(Cast(Literal(date), DateType, gmtId)), 2) - checkEvaluation(WeekDay(Cast(Literal(ts), DateType, gmtId)), 4) - checkEvaluation(WeekDay(Cast(Literal("2011-05-06"), DateType, gmtId)), 4) + checkEvaluation(WeekDay(Cast(Literal(date), DateType, UTC_OPT)), 2) + checkEvaluation(WeekDay(Cast(Literal(ts), DateType, UTC_OPT)), 4) + checkEvaluation(WeekDay(Cast(Literal("2011-05-06"), DateType, UTC_OPT)), 4) checkEvaluation(WeekDay(Literal(new Date(toMillis("2017-05-27 13:10:15")))), 5) checkEvaluation(WeekDay(Literal(new Date(toMillis("1582-10-15 13:10:15")))), 4) checkConsistencyBetweenInterpretedAndCodegen(WeekDay, DateType) @@ -232,11 +227,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("WeekOfYear") { checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) checkEvaluation(WeekOfYear(Literal(d)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(date), DateType, gmtId)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType, gmtId)), 45) - checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType, gmtId)), 18) - checkEvaluation(WeekOfYear(Cast(Literal("1582-10-15 13:10:15"), DateType, gmtId)), 41) - checkEvaluation(WeekOfYear(Cast(Literal("1582-10-04 13:10:15"), DateType, gmtId)), 40) + checkEvaluation(WeekOfYear(Cast(Literal(date), DateType, UTC_OPT)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType, UTC_OPT)), 45) + checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType, UTC_OPT)), 18) + checkEvaluation(WeekOfYear(Cast(Literal("1582-10-15 13:10:15"), DateType, UTC_OPT)), 41) + checkEvaluation(WeekOfYear(Cast(Literal("1582-10-04 13:10:15"), DateType, UTC_OPT)), 40) checkConsistencyBetweenInterpretedAndCodegen(WeekOfYear, DateType) } @@ -244,38 +239,42 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { Seq("legacy", "corrected").foreach { legacyParserPolicy => withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy) { checkEvaluation( - DateFormatClass(Literal.create(null, TimestampType), Literal("y"), gmtId), + DateFormatClass(Literal.create(null, TimestampType), Literal("y"), UTC_OPT), 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") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), + Literal.create(null, StringType), UTC_OPT), null) + + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), + Literal("y"), UTC_OPT), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), UTC_OPT), "2013") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), + Literal("H"), UTC_OPT), "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), UTC_OPT), "13") + + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), + Literal("y"), PST_OPT), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), PST_OPT), "2013") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), + Literal("H"), PST_OPT), "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), PST_OPT), "5") + + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), + Literal("y"), JST_OPT), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), JST_OPT), "2013") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), + Literal("H"), JST_OPT), "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), JST_OPT), "22") + + // Test escaping of format + GenerateUnsafeProjection.generate( + DateFormatClass(Literal(ts), Literal("\"quote"), JST_OPT) :: Nil) // SPARK-28072 The codegen path should work checkEvaluation( expression = DateFormatClass( BoundReference(ordinal = 0, dataType = TimestampType, nullable = true), BoundReference(ordinal = 1, dataType = StringType, nullable = true), - jstId), + JST_OPT), expected = "22", inputRow = InternalRow(DateTimeUtils.fromJavaTimestamp(ts), UTF8String.fromString("H"))) } @@ -283,16 +282,16 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Hour") { - assert(Hour(Literal.create(null, DateType), gmtId).resolved === false) - assert(Hour(Literal(ts), gmtId).resolved) - checkEvaluation(Hour(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) - checkEvaluation(Hour(Cast(Literal(date), TimestampType, gmtId), gmtId), 13) - checkEvaluation(Hour(Literal(ts), gmtId), 13) + assert(Hour(Literal.create(null, DateType), UTC_OPT).resolved === false) + assert(Hour(Literal(ts), UTC_OPT).resolved) + checkEvaluation(Hour(Cast(Literal(d), TimestampType, UTC_OPT), UTC_OPT), 0) + checkEvaluation(Hour(Cast(Literal(date), TimestampType, UTC_OPT), UTC_OPT), 13) + checkEvaluation(Hour(Literal(ts), UTC_OPT), 13) val c = Calendar.getInstance() - for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { - val timeZoneId = Option(tz.getID) - c.setTimeZone(tz) + for (zid <- outstandingZoneIds) { + val timeZoneId = Option(zid.getId) + c.setTimeZone(TimeZone.getTimeZone(zid)) (0 to 24 by 6).foreach { h => (0 to 60 by 30).foreach { m => (0 to 60 by 30).foreach { s => @@ -309,17 +308,17 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Minute") { - assert(Minute(Literal.create(null, DateType), gmtId).resolved === false) - assert(Minute(Literal(ts), gmtId).resolved) - checkEvaluation(Minute(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) + assert(Minute(Literal.create(null, DateType), UTC_OPT).resolved === false) + assert(Minute(Literal(ts), UTC_OPT).resolved) + checkEvaluation(Minute(Cast(Literal(d), TimestampType, UTC_OPT), UTC_OPT), 0) checkEvaluation( - Minute(Cast(Literal(date), TimestampType, gmtId), gmtId), 10) - checkEvaluation(Minute(Literal(ts), gmtId), 10) + Minute(Cast(Literal(date), TimestampType, UTC_OPT), UTC_OPT), 10) + checkEvaluation(Minute(Literal(ts), UTC_OPT), 10) val c = Calendar.getInstance() - for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { - val timeZoneId = Option(tz.getID) - c.setTimeZone(tz) + for (zid <- outstandingZoneIds) { + val timeZoneId = Option(zid.getId) + c.setTimeZone(TimeZone.getTimeZone(zid)) (0 to 60 by 5).foreach { m => (0 to 60 by 15).foreach { s => c.set(2015, 18, 3, 3, m, s) @@ -389,9 +388,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("time_add") { 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) + for (zid <- outstandingZoneIds) { + val timeZoneId = Option(zid.getId) + sdf.setTimeZone(TimeZone.getTimeZone(zid)) checkEvaluation( TimeAdd( @@ -427,9 +426,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("time_sub") { 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) + for (zid <- outstandingZoneIds) { + val timeZoneId = Option(zid.getId) + sdf.setTimeZone(TimeZone.getTimeZone(zid)) checkEvaluation( TimeSub( @@ -513,9 +512,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("months_between") { 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) + for (zid <- outstandingZoneIds) { + val timeZoneId = Option(zid.getId) + sdf.setTimeZone(TimeZone.getTimeZone(zid)) checkEvaluation( MonthsBetween( @@ -607,6 +606,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(NextDay(Literal.create(null, DateType), Literal("xx")), null) checkEvaluation( NextDay(Literal(Date.valueOf("2015-07-23")), Literal.create(null, StringType)), null) + // Test escaping of dayOfWeek + GenerateUnsafeProjection.generate( + NextDay(Literal(Date.valueOf("2015-07-23")), Literal("\"quote")) :: Nil) } test("TruncDate") { @@ -630,6 +632,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { testTrunc(date, null, null) testTrunc(null, "MON", null) testTrunc(null, null, null) + // Test escaping of format + GenerateUnsafeProjection.generate(TruncDate(Literal(0, DateType), Literal("\"quote")) :: Nil) testTrunc(Date.valueOf("2000-03-08"), "decade", Date.valueOf("2000-01-01")) testTrunc(Date.valueOf("2000-03-08"), "century", Date.valueOf("1901-01-01")) @@ -646,7 +650,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { expected) } - withDefaultTimeZone(TimeZoneGMT) { + withDefaultTimeZone(UTC) { val inputDate = Timestamp.valueOf("2015-07-22 05:30:06") Seq("yyyy", "YYYY", "year", "YEAR", "yy", "YY").foreach { fmt => @@ -716,8 +720,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val sdf1 = new SimpleDateFormat(fmt1, Locale.US) val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" val sdf2 = new SimpleDateFormat(fmt2, Locale.US) - for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { - val timeZoneId = Option(tz.getID) + for (zid <- outstandingZoneIds) { + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) sdf1.setTimeZone(tz) sdf2.setTimeZone(tz) @@ -755,6 +760,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } } + // Test escaping of format + GenerateUnsafeProjection.generate(FromUnixTime(Literal(0L), Literal("\"quote")) :: Nil) } test("unix_timestamp") { @@ -765,11 +772,12 @@ 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) - sdf3.setTimeZone(TimeZoneGMT) + sdf3.setTimeZone(TimeZone.getTimeZone(UTC)) - withDefaultTimeZone(TimeZoneGMT) { - for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { - val timeZoneId = Option(tz.getID) + withDefaultTimeZone(UTC) { + for (zid <- outstandingZoneIds) { + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) sdf1.setTimeZone(tz) sdf2.setTimeZone(tz) @@ -821,6 +829,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } } + // Test escaping of format + GenerateUnsafeProjection.generate( + UnixTimestamp(Literal("2015-07-24"), Literal("\"quote")) :: Nil) } test("to_unix_timestamp") { @@ -832,11 +843,12 @@ 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) - sdf3.setTimeZone(TimeZoneGMT) + sdf3.setTimeZone(TimeZone.getTimeZone(UTC)) - withDefaultTimeZone(TimeZoneGMT) { - for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { - val timeZoneId = Option(tz.getID) + withDefaultTimeZone(UTC) { + for (zid <- outstandingZoneIds) { + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) sdf1.setTimeZone(tz) sdf2.setTimeZone(tz) @@ -852,7 +864,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), MICROSECONDS.toSeconds( - DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid))) checkEvaluation( ToUnixTimestamp( Literal(sdf2.format(new Timestamp(-1000000))), @@ -861,7 +873,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(ToUnixTimestamp( Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), MICROSECONDS.toSeconds(DateTimeUtils.daysToMicros( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz.toZoneId))) + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), zid))) val t1 = ToUnixTimestamp( CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] val t2 = ToUnixTimestamp( @@ -876,7 +888,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(ToUnixTimestamp( Literal(date1), Literal.create(null, StringType), timeZoneId), MICROSECONDS.toSeconds( - DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid))) checkEvaluation( ToUnixTimestamp( Literal("2015-07-24"), @@ -895,6 +907,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } } + // Test escaping of format + GenerateUnsafeProjection.generate( + ToUnixTimestamp(Literal("2015-07-24"), Literal("\"quote")) :: Nil) } test("datediff") { @@ -924,8 +939,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { NonFoldableLiteral.create(tz, StringType)), if (expected != null) Timestamp.valueOf(expected) else null) } - test("2015-07-24 00:00:00", "PST", "2015-07-24 07:00:00") - test("2015-01-24 00:00:00", "PST", "2015-01-24 08:00:00") + test("2015-07-24 00:00:00", LA.getId, "2015-07-24 07:00:00") + test("2015-01-24 00:00:00", LA.getId, "2015-01-24 08:00:00") test(null, "UTC", null) test("2015-07-24 00:00:00", null, null) test(null, null, null) @@ -955,8 +970,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { NonFoldableLiteral.create(tz, StringType)), if (expected != null) Timestamp.valueOf(expected) else null) } - test("2015-07-24 00:00:00", "PST", "2015-07-23 17:00:00") - test("2015-01-24 00:00:00", "PST", "2015-01-23 16:00:00") + test("2015-07-24 00:00:00", LA.getId, "2015-07-23 17:00:00") + test("2015-01-24 00:00:00", LA.getId, "2015-01-23 16:00:00") test(null, "UTC", null) test("2015-07-24 00:00:00", null, null) test(null, null, null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index 3693531f47610..6f062dcc9a1ce 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -18,15 +18,17 @@ package org.apache.spark.sql.catalyst.expressions import java.text.{DecimalFormat, DecimalFormatSymbols, SimpleDateFormat} -import java.util.{Calendar, Locale} +import java.util.{Calendar, Locale, TimeZone} import org.scalatest.exceptions.TestFailedException import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{PST, UTC, UTC_OPT} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -48,6 +50,10 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with in Jackson's JsonFactory.createParser(byte[]) due to RFC-4627 encoding detection */ val badJson = "\u0000\u0000\u0000A\u0001AAA" + test("get_json_object escaping") { + GenerateUnsafeProjection.generate(GetJsonObject(Literal("\"quote"), Literal("\"quote")) :: Nil) + } + test("$.store.bicycle") { checkEvaluation( GetJsonObject(Literal(json), Literal("$.store.bicycle")), @@ -264,6 +270,11 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with assert(jt.eval(null).toSeq.head === expected) } + test("json_tuple escaping") { + GenerateUnsafeProjection.generate( + JsonTuple(Literal("\"quote") :: Literal("\"quote") :: Nil) :: Nil) + } + test("json_tuple - hive key 1") { checkJsonTuple( JsonTuple( @@ -395,13 +406,17 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with InternalRow(UTF8String.fromString("1"), null, UTF8String.fromString("1"))) } - val gmtId = Option(DateTimeUtils.TimeZoneGMT.getID) + test("from_json escaping") { + val schema = StructType(StructField("\"quote", IntegerType) :: Nil) + GenerateUnsafeProjection.generate( + JsonToStructs(schema, Map.empty, Literal("\"quote"), UTC_OPT) :: Nil) + } test("from_json") { val jsonData = """{"a": 1}""" val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( - JsonToStructs(schema, Map.empty, Literal(jsonData), gmtId), + JsonToStructs(schema, Map.empty, Literal(jsonData), UTC_OPT), InternalRow(1) ) } @@ -410,13 +425,13 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val jsonData = """{"a" 1}""" val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( - JsonToStructs(schema, Map.empty, Literal(jsonData), gmtId), + JsonToStructs(schema, Map.empty, Literal(jsonData), UTC_OPT), InternalRow(null) ) val exception = intercept[TestFailedException] { checkEvaluation( - JsonToStructs(schema, Map("mode" -> FailFastMode.name), Literal(jsonData), gmtId), + JsonToStructs(schema, Map("mode" -> FailFastMode.name), Literal(jsonData), UTC_OPT), InternalRow(null) ) }.getCause @@ -429,35 +444,35 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val input = """[{"a": 1}, {"a": 2}]""" val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) val output = InternalRow(1) :: InternalRow(2) :: Nil - checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) + checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output) } test("from_json - input=object, schema=array, output=array of single row") { val input = """{"a": 1}""" val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) val output = InternalRow(1) :: Nil - checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) + checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output) } test("from_json - input=empty array, schema=array, output=empty array") { val input = "[ ]" val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) val output = Nil - checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) + checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output) } test("from_json - input=empty object, schema=array, output=array of single row with null") { val input = "{ }" val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) val output = InternalRow(null) :: Nil - checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) + checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output) } test("from_json - input=array of single object, schema=struct, output=single row") { val input = """[{"a": 1}]""" val schema = StructType(StructField("a", IntegerType) :: Nil) val output = InternalRow(null) - checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) + checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output) } test("from_json - input=array, schema=struct, output=single row") { @@ -466,27 +481,27 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val schema = new StructType().add("a", IntegerType).add(corrupted, StringType) val output = InternalRow(null, UTF8String.fromString(input)) val options = Map("columnNameOfCorruptRecord" -> corrupted) - checkEvaluation(JsonToStructs(schema, options, Literal(input), gmtId), output) + checkEvaluation(JsonToStructs(schema, options, Literal(input), UTC_OPT), output) } test("from_json - input=empty array, schema=struct, output=single row with null") { val input = """[]""" val schema = StructType(StructField("a", IntegerType) :: Nil) val output = InternalRow(null) - checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) + checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output) } test("from_json - input=empty object, schema=struct, output=single row with null") { val input = """{ }""" val schema = StructType(StructField("a", IntegerType) :: Nil) val output = InternalRow(null) - checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) + checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output) } test("from_json null input column") { val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( - JsonToStructs(schema, Map.empty, Literal.create(null, StringType), gmtId), + JsonToStructs(schema, Map.empty, Literal.create(null, StringType), UTC_OPT), null ) } @@ -494,7 +509,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("SPARK-20549: from_json bad UTF-8") { val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( - JsonToStructs(schema, Map.empty, Literal(badJson), gmtId), + JsonToStructs(schema, Map.empty, Literal(badJson), UTC_OPT), InternalRow(null)) } @@ -502,24 +517,24 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val schema = StructType(StructField("t", TimestampType) :: Nil) val jsonData1 = """{"t": "2016-01-01T00:00:00.123Z"}""" - var c = Calendar.getInstance(DateTimeUtils.TimeZoneGMT) + var c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) c.set(2016, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 123) checkEvaluation( - JsonToStructs(schema, Map.empty, Literal(jsonData1), gmtId), + JsonToStructs(schema, Map.empty, Literal(jsonData1), UTC_OPT), InternalRow(c.getTimeInMillis * 1000L) ) // The result doesn't change because the json string includes timezone string ("Z" here), // which means the string represents the timestamp string in the timezone regardless of // the timeZoneId parameter. checkEvaluation( - JsonToStructs(schema, Map.empty, Literal(jsonData1), Option("PST")), + JsonToStructs(schema, Map.empty, Literal(jsonData1), Option(PST.getId)), InternalRow(c.getTimeInMillis * 1000L) ) val jsonData2 = """{"t": "2016-01-01T00:00:00"}""" - for (tz <- DateTimeTestUtils.outstandingTimezones) { - c = Calendar.getInstance(tz) + for (zid <- DateTimeTestUtils.outstandingZoneIds) { + c = Calendar.getInstance(TimeZone.getTimeZone(zid)) c.set(2016, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) checkEvaluation( @@ -527,16 +542,16 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with schema, Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss"), Literal(jsonData2), - Option(tz.getID)), + Option(zid.getId)), InternalRow(c.getTimeInMillis * 1000L) ) checkEvaluation( JsonToStructs( schema, Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", - DateTimeUtils.TIMEZONE_OPTION -> tz.getID), + DateTimeUtils.TIMEZONE_OPTION -> zid.getId), Literal(jsonData2), - gmtId), + UTC_OPT), InternalRow(c.getTimeInMillis * 1000L) ) } @@ -545,16 +560,23 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("SPARK-19543: from_json empty input column") { val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( - JsonToStructs(schema, Map.empty, Literal.create(" ", StringType), gmtId), + JsonToStructs(schema, Map.empty, Literal.create(" ", StringType), UTC_OPT), null ) } + test("to_json escaping") { + val schema = StructType(StructField("\"quote", IntegerType) :: Nil) + val struct = Literal.create(create_row(1), schema) + GenerateUnsafeProjection.generate( + StructsToJson(Map.empty, struct, UTC_OPT) :: Nil) + } + test("to_json - struct") { val schema = StructType(StructField("a", IntegerType) :: Nil) val struct = Literal.create(create_row(1), schema) checkEvaluation( - StructsToJson(Map.empty, struct, gmtId), + StructsToJson(Map.empty, struct, UTC_OPT), """{"a":1}""" ) } @@ -564,7 +586,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val input = new GenericArrayData(InternalRow(1) :: InternalRow(2) :: Nil) val output = """[{"a":1},{"a":2}]""" checkEvaluation( - StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId), + StructsToJson(Map.empty, Literal.create(input, inputSchema), UTC_OPT), output) } @@ -573,7 +595,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val input = new GenericArrayData(InternalRow(null) :: Nil) val output = """[{}]""" checkEvaluation( - StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId), + StructsToJson(Map.empty, Literal.create(input, inputSchema), UTC_OPT), output) } @@ -582,7 +604,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val input = new GenericArrayData(Nil) val output = """[]""" checkEvaluation( - StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId), + StructsToJson(Map.empty, Literal.create(input, inputSchema), UTC_OPT), output) } @@ -590,41 +612,41 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val schema = StructType(StructField("a", IntegerType) :: Nil) val struct = Literal.create(null, schema) checkEvaluation( - StructsToJson(Map.empty, struct, gmtId), + StructsToJson(Map.empty, struct, UTC_OPT), null ) } test("to_json with timestamp") { val schema = StructType(StructField("t", TimestampType) :: Nil) - val c = Calendar.getInstance(DateTimeUtils.TimeZoneGMT) + val c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) c.set(2016, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) val struct = Literal.create(create_row(c.getTimeInMillis * 1000L), schema) checkEvaluation( - StructsToJson(Map.empty, struct, gmtId), + StructsToJson(Map.empty, struct, UTC_OPT), """{"t":"2016-01-01T00:00:00.000Z"}""" ) checkEvaluation( - StructsToJson(Map.empty, struct, Option("PST")), + StructsToJson(Map.empty, struct, Option(PST.getId)), """{"t":"2015-12-31T16:00:00.000-08:00"}""" ) checkEvaluation( StructsToJson( Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", - DateTimeUtils.TIMEZONE_OPTION -> gmtId.get), + DateTimeUtils.TIMEZONE_OPTION -> UTC_OPT.get), struct, - gmtId), + UTC_OPT), """{"t":"2016-01-01T00:00:00"}""" ) checkEvaluation( StructsToJson( Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", - DateTimeUtils.TIMEZONE_OPTION -> "PST"), + DateTimeUtils.TIMEZONE_OPTION -> PST.getId), struct, - gmtId), + UTC_OPT), """{"t":"2015-12-31T16:00:00"}""" ) } @@ -665,7 +687,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with ArrayBasedMapData(Map(UTF8String.fromString("b") -> 2)) :: Nil) val output = """[{"a":1},{"b":2}]""" checkEvaluation( - StructsToJson(Map.empty, Literal(input, inputSchema), gmtId), + StructsToJson(Map.empty, Literal(input, inputSchema), UTC_OPT), output) } @@ -674,7 +696,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val input = new GenericArrayData(ArrayBasedMapData(Map(UTF8String.fromString("a") -> 1)) :: Nil) val output = """[{"a":1}]""" checkEvaluation( - StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId), + StructsToJson(Map.empty, Literal.create(input, inputSchema), UTC_OPT), output) } @@ -704,7 +726,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with .add("b", StringType, nullable = false) .add("c", StringType, nullable = false) val output = InternalRow(1L, null, UTF8String.fromString("foo")) - val expr = JsonToStructs(jsonSchema, Map.empty, Literal.create(input, StringType), gmtId) + val expr = JsonToStructs(jsonSchema, Map.empty, Literal.create(input, StringType), UTC_OPT) checkEvaluation(expr, output) val schema = expr.dataType val schemaToCompare = jsonSchema.asNullable @@ -737,7 +759,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val options = Map("dateFormat" -> dateFormat, "locale" -> langTag) checkEvaluation( - JsonToStructs(schema, options, Literal.create(dateStr), gmtId), + JsonToStructs(schema, options, Literal.create(dateStr), UTC_OPT), InternalRow(17836)) // number of days from 1970-01-01 } } @@ -748,7 +770,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with schema = StructType.fromDDL("i int, _unparsed boolean"), options = Map("columnNameOfCorruptRecord" -> "_unparsed"), child = Literal.create("""{"i":"a"}"""), - timeZoneId = gmtId), + timeZoneId = UTC_OPT), expectedErrMsg = "The field for corrupt records must be string type and nullable") } @@ -770,7 +792,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val (expected, input) = decimalInput(langTag) checkEvaluation( - JsonToStructs(schema, options, Literal.create(input), gmtId), + JsonToStructs(schema, options, Literal.create(input), UTC_OPT), InternalRow(expected)) } @@ -791,4 +813,44 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with checkDecimalInfer(_, """struct""") } } + + test("Length of JSON array") { + Seq( + ("", null), + ("[1,2,3]", 3), + ("[]", 0), + ("[[1],[2,3],[]]", 3), + ("""[{"a":123},{"b":"hello"}]""", 2), + ("""[1,2,3,[33,44],{"key":[2,3,4]}]""", 5), + ("""[1,2,3,4,5""", null), + ("Random String", null), + ("""{"key":"not a json array"}""", null), + ("""{"key": 25}""", null) + ).foreach { + case(literal, expectedValue) => + checkEvaluation(LengthOfJsonArray(Literal(literal)), expectedValue) + } + } + + test("json_object_keys") { + Seq( + // Invalid inputs + ("", null), + ("[]", null), + ("""[{"key": "JSON"}]""", null), + ("""{"key": 45, "random_string"}""", null), + ("""{[1, 2, {"Key": "Invalid JSON"}]}""", null), + // JSON objects + ("{}", Seq.empty[UTF8String]), + ("""{"key": 1}""", Seq("key")), + ("""{"key": "value", "key2": 2}""", Seq("key", "key2")), + ("""{"arrayKey": [1, 2, 3]}""", Seq("arrayKey")), + ("""{"key":[1,2,3,{"key":"value"},[1,2,3]]}""", Seq("key")), + ("""{"f1":"abc","f2":{"f3":"a", "f4":"b"}}""", Seq("f1", "f2")), + ("""{"k1": [1, 2, {"key": 5}], "k2": {"key2": [1, 2]}}""", Seq("k1", "k2")) + ).foreach { + case (input, expected) => + checkEvaluation(JsonObjectKeys(Literal(input)), expected) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index 712d2bc4c4736..ad9492a8d3ab1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.types.StringType /** @@ -255,6 +256,10 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val nonNullExpr = RegExpReplace(Literal("100-200"), Literal("(\\d+)"), Literal("num")) checkEvaluation(nonNullExpr, "num-num", row1) + + // Test escaping of arguments + GenerateUnsafeProjection.generate( + RegExpReplace(Literal("\"quote"), Literal("\"quote"), Literal("\"quote")) :: Nil) } test("SPARK-22570: RegExpReplace should not create a lot of global variables") { @@ -305,6 +310,10 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { expr, row9, "Regex group count is 1, but the specified group index is 2") checkExceptionInExpression[IllegalArgumentException]( expr, row10, "Regex group count is 0, but the specified group index is 1") + + // Test escaping of arguments + GenerateUnsafeProjection.generate( + RegExpExtract(Literal("\"quote"), Literal("\"quote"), Literal(1)) :: Nil) } test("SPLIT") { @@ -327,6 +336,10 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { StringSplit(s1, s2, -1), Seq("aa", "bb", "cc"), row1) checkEvaluation(StringSplit(s1, s2, -1), null, row2) checkEvaluation(StringSplit(s1, s2, -1), null, row3) + + // Test escaping of arguments + GenerateUnsafeProjection.generate( + StringSplit(Literal("\"quote"), Literal("\"quote"), Literal(-1)) :: Nil) } test("SPARK-30759: cache initialization for literal patterns") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index c5ffc381b58e2..836b2eaa642a3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Locale import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DecimalType, IntegerType, StringType} @@ -27,10 +28,12 @@ import org.apache.spark.sql.types.{DecimalType, IntegerType, StringType} class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { test("basic") { - val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil, true :: Nil) + val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil, + Option(ExpressionEncoder[Int]()) :: Nil) checkEvaluation(intUdf, 2) - val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, false :: Nil) + val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, + Option(ExpressionEncoder[String]()) :: Nil) checkEvaluation(stringUdf, "ax") } @@ -39,7 +42,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { (s: String) => s.toLowerCase(Locale.ROOT), StringType, Literal.create(null, StringType) :: Nil, - false :: Nil) + Option(ExpressionEncoder[String]()) :: Nil) val e1 = intercept[SparkException](udf.eval()) assert(e1.getMessage.contains("Failed to execute user defined function")) @@ -52,7 +55,8 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-22695: ScalaUDF should not use global variables") { val ctx = new CodegenContext - ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, false :: Nil).genCode(ctx) + ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, + Option(ExpressionEncoder[String]()) :: Nil).genCode(ctx) assert(ctx.inlinedMutableStates.isEmpty) } @@ -61,7 +65,8 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, - Literal(BigDecimal("12345678901234567890.123")) :: Nil, false :: Nil) + Literal(BigDecimal("12345678901234567890.123")) :: Nil, + Option(ExpressionEncoder[java.math.BigDecimal]()) :: Nil) val e1 = intercept[ArithmeticException](udf.eval()) assert(e1.getMessage.contains("cannot be represented as Decimal")) val e2 = intercept[SparkException] { @@ -73,7 +78,8 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, - Literal(BigDecimal("12345678901234567890.123")) :: Nil, false :: Nil) + Literal(BigDecimal("12345678901234567890.123")) :: Nil, + Option(ExpressionEncoder[java.math.BigDecimal]()) :: Nil) checkEvaluation(udf, null) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 4308f98d6969a..f18364d844ce1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.types._ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -155,6 +156,11 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(c1 endsWith "b", false, row) checkEvaluation(c2 endsWith "b", null, row) checkEvaluation(c1 endsWith Literal.create(null, StringType), null, row) + + // Test escaping of arguments + GenerateUnsafeProjection.generate(Contains(Literal("\"quote"), Literal("\"quote")) :: Nil) + GenerateUnsafeProjection.generate(EndsWith(Literal("\"quote"), Literal("\"quote")) :: Nil) + GenerateUnsafeProjection.generate(StartsWith(Literal("\"quote"), Literal("\"quote")) :: Nil) } test("Substring") { @@ -352,6 +358,10 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Decode(b, Literal("utf-8")), null, create_row(null)) checkEvaluation(Decode(Literal.create(null, BinaryType), Literal("utf-8")), null) checkEvaluation(Decode(b, Literal.create(null, StringType)), null, create_row(null)) + + // Test escaping of charset + GenerateUnsafeProjection.generate(Encode(a, Literal("\"quote")) :: Nil) + GenerateUnsafeProjection.generate(Decode(b, Literal("\"quote")) :: Nil) } test("initcap unit test") { @@ -379,6 +389,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Levenshtein(Literal("千世"), Literal("fog")), 3) checkEvaluation(Levenshtein(Literal("世界千世"), Literal("大a界b")), 4) // scalastyle:on + + // Test escaping of arguments: + GenerateUnsafeProjection.generate(Levenshtein(Literal("\"quotea"), Literal("\"quoteb")) :: Nil) } test("soundex unit test") { @@ -560,6 +573,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(StringTrim(Literal("a"), Literal.create(null, StringType)), null) checkEvaluation(StringTrim(Literal.create(null, StringType), Literal("a")), null) + // Test escaping of arguments + GenerateUnsafeProjection.generate(StringTrim(Literal("\"quote"), Literal("\"quote")) :: Nil) + checkEvaluation(StringTrim(Literal("yxTomxx"), Literal("xyz")), "Tom") checkEvaluation(StringTrim(Literal("xxxbarxxx"), Literal("x")), "bar") } @@ -587,6 +603,10 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(StringTrimLeft(Literal.create(null, StringType), Literal("a")), null) checkEvaluation(StringTrimLeft(Literal("a"), Literal.create(null, StringType)), null) + // Test escaping of arguments + GenerateUnsafeProjection.generate( + StringTrimLeft(Literal("\"quote"), Literal("\"quote")) :: Nil) + checkEvaluation(StringTrimLeft(Literal("zzzytest"), Literal("xyz")), "test") checkEvaluation(StringTrimLeft(Literal("zzzytestxyz"), Literal("xyz")), "testxyz") checkEvaluation(StringTrimLeft(Literal("xyxXxyLAST WORD"), Literal("xy")), "XxyLAST WORD") @@ -616,6 +636,10 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(StringTrimRight(Literal("a"), Literal.create(null, StringType)), null) checkEvaluation(StringTrimRight(Literal.create(null, StringType), Literal("a")), null) + // Test escaping of arguments + GenerateUnsafeProjection.generate( + StringTrimRight(Literal("\"quote"), Literal("\"quote")) :: Nil) + checkEvaluation(StringTrimRight(Literal("testxxzx"), Literal("xyz")), "test") checkEvaluation(StringTrimRight(Literal("xyztestxxzx"), Literal("xyz")), "xyztest") checkEvaluation(StringTrimRight(Literal("TURNERyxXxy"), Literal("xy")), "TURNERyxX") @@ -632,6 +656,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { FormatString(Literal("aa%d%s"), Literal.create(null, IntegerType), "cc"), "aanullcc") checkEvaluation( FormatString(Literal("aa%d%s"), 12, Literal.create(null, StringType)), "aa12null") + + // Test escaping of arguments + GenerateUnsafeProjection.generate(FormatString(Literal("\"quote"), Literal("\"quote")) :: Nil) } test("SPARK-22603: FormatString should not generate codes beyond 64KB") { @@ -662,6 +689,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(StringInstr(s1, s2), 1, create_row("花花世界", "花")) checkEvaluation(StringInstr(s1, s2), 0, create_row("花花世界", "小")) // scalastyle:on + + // Test escaping of arguments + GenerateUnsafeProjection.generate(StringInstr(Literal("\"quote"), Literal("\"quote")) :: Nil) } test("LOCATE") { @@ -718,6 +748,10 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(StringRPad(s1, s2, s3), null, row3) checkEvaluation(StringRPad(s1, s2, s3), null, row4) checkEvaluation(StringRPad(s1, s2, s3), null, row5) + + // Test escaping of arguments + GenerateUnsafeProjection.generate(StringLPad(Literal("\"quote"), s2, Literal("\"quote")) :: Nil) + GenerateUnsafeProjection.generate(StringRPad(Literal("\"quote"), s2, Literal("\"quote")) :: Nil) checkEvaluation(StringRPad(Literal("hi"), Literal(5)), "hi ") checkEvaluation(StringRPad(Literal("hi"), Literal(1)), "h") } @@ -732,6 +766,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(StringRepeat(Literal("hi"), Literal(-1)), "", row1) checkEvaluation(StringRepeat(s1, s2), "hihi", row1) checkEvaluation(StringRepeat(s1, s2), null, row2) + + // Test escaping of arguments + GenerateUnsafeProjection.generate(StringRepeat(Literal("\"quote"), Literal(2)) :: Nil) } test("REVERSE") { @@ -897,6 +934,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(ParseUrl(Seq(Literal("1"), Literal(2))).checkInputDataTypes().isFailure) assert(ParseUrl(Seq(Literal(1), Literal("2"))).checkInputDataTypes().isFailure) assert(ParseUrl(Seq(Literal("1"), Literal("2"), Literal(3))).checkInputDataTypes().isFailure) + + // Test escaping of arguments + GenerateUnsafeProjection.generate(ParseUrl(Seq(Literal("\"quote"), Literal("\"quote"))) :: Nil) } test("Sentences") { @@ -919,5 +959,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { answer) checkEvaluation(Sentences("Hi there! The price was $1,234.56.... But, not now.", "XXX", "YYY"), answer) + + // Test escaping of arguments + GenerateUnsafeProjection.generate( + Sentences(Literal("\"quote"), Literal("\"quote"), Literal("\"quote")) :: Nil) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala index d9a6fbf81de91..d7eb048ba8705 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry} import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -244,7 +245,8 @@ class EliminateSortsSuite extends PlanTest { } test("should not remove orderBy in groupBy clause with ScalaUDF as aggs") { - val scalaUdf = ScalaUDF((s: Int) => s, IntegerType, 'a :: Nil, true :: Nil) + val scalaUdf = ScalaUDF((s: Int) => s, IntegerType, 'a :: Nil, + Option(ExpressionEncoder[Int]()) :: Nil) val projectPlan = testRelation.select('a, 'b) val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc) val groupByPlan = orderByPlan.groupBy('a)(scalaUdf) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index a9267379e12fd..d4d6f79d7895e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -261,6 +261,86 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { .analyze comparePlans(optimized, expected) } + + test("Nested field pruning for Project and Generate") { + val query = contact + .generate(Explode('friends.getField("first")), outputNames = Seq("explode")) + .select('explode, 'friends.getField("middle")) + .analyze + val optimized = Optimize.execute(query) + + val aliases = collectGeneratedAliases(optimized) + + val expected = contact + .select( + 'friends.getField("middle").as(aliases(0)), + 'friends.getField("first").as(aliases(1))) + .generate(Explode($"${aliases(1)}"), + unrequiredChildIndex = Seq(1), + outputNames = Seq("explode")) + .select('explode, $"${aliases(0)}".as("friends.middle")) + .analyze + comparePlans(optimized, expected) + } + + test("Nested field pruning for Generate") { + val query = contact + .generate(Explode('friends.getField("first")), outputNames = Seq("explode")) + .select('explode) + .analyze + val optimized = Optimize.execute(query) + + val aliases = collectGeneratedAliases(optimized) + + val expected = contact + .select('friends.getField("first").as(aliases(0))) + .generate(Explode($"${aliases(0)}"), + unrequiredChildIndex = Seq(0), + outputNames = Seq("explode")) + .analyze + comparePlans(optimized, expected) + } + + test("Nested field pruning for Project and Generate: not prune on generator output") { + val companies = LocalRelation( + 'id.int, + 'employers.array(employer)) + + val query = companies + .generate(Explode('employers.getField("company")), outputNames = Seq("company")) + .select('company.getField("name")) + .analyze + val optimized = Optimize.execute(query) + + val aliases = collectGeneratedAliases(optimized) + + val expected = companies + .select('employers.getField("company").as(aliases(0))) + .generate(Explode($"${aliases(0)}"), + unrequiredChildIndex = Seq(0), + outputNames = Seq("company")) + .select('company.getField("name").as("company.name")) + .analyze + comparePlans(optimized, expected) + } + + test("Nested field pruning for Generate: not prune on required child output") { + val query = contact + .generate( + Explode('friends.getField("first")), + outputNames = Seq("explode")) + .select('explode, 'friends) + .analyze + val optimized = Optimize.execute(query) + + val expected = contact + .select('friends) + .generate(Explode('friends.getField("first")), + outputNames = Seq("explode")) + .select('explode, 'friends) + .analyze + comparePlans(optimized, expected) + } } object NestedColumnAliasingSuite { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala index f00d22e6e96a6..0608ded739374 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.ListQuery -import org.apache.spark.sql.catalyst.plans.{LeftSemi, PlanTest} +import org.apache.spark.sql.catalyst.expressions.{IsNull, ListQuery, Not} +import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, LeftSemi, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -52,4 +52,19 @@ class RewriteSubquerySuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("NOT-IN subquery nested inside OR") { + val relation1 = LocalRelation('a.int, 'b.int) + val relation2 = LocalRelation('c.int, 'd.int) + val exists = 'exists.boolean.notNull + + val query = relation1.where('b === 1 || Not('a.in(ListQuery(relation2.select('c))))).select('a) + val correctAnswer = relation1 + .join(relation2.select('c), ExistenceJoin(exists), Some('a === 'c || IsNull('a === 'c))) + .where('b === 1 || Not(exists)) + .select('a) + .analyze + val optimized = Optimize.execute(query.analyze) + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 7ddb5941c2651..572ea840eb88b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition.{after, first} import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -1224,6 +1225,12 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("SHOW TABLES"), ShowTables(UnresolvedNamespace(Seq.empty[String]), None)) + comparePlans( + parsePlan("SHOW TABLES '*test*'"), + ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*"))) + comparePlans( + parsePlan("SHOW TABLES LIKE '*test*'"), + ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*"))) comparePlans( parsePlan("SHOW TABLES FROM testcat.ns1.ns2.tbl"), ShowTables(UnresolvedNamespace(Seq("testcat", "ns1", "ns2", "tbl")), None)) @@ -1231,8 +1238,11 @@ class DDLParserSuite extends AnalysisTest { parsePlan("SHOW TABLES IN testcat.ns1.ns2.tbl"), ShowTables(UnresolvedNamespace(Seq("testcat", "ns1", "ns2", "tbl")), None)) comparePlans( - parsePlan("SHOW TABLES IN tbl LIKE '*dog*'"), - ShowTables(UnresolvedNamespace(Seq("tbl")), Some("*dog*"))) + parsePlan("SHOW TABLES IN ns1 '*test*'"), + ShowTables(UnresolvedNamespace(Seq("ns1")), Some("*test*"))) + comparePlans( + parsePlan("SHOW TABLES IN ns1 LIKE '*test*'"), + ShowTables(UnresolvedNamespace(Seq("ns1")), Some("*test*"))) } test("show table extended") { @@ -1260,6 +1270,30 @@ class DDLParserSuite extends AnalysisTest { Some(Map("ds" -> "2008-04-09")))) } + test("show views") { + comparePlans( + parsePlan("SHOW VIEWS"), + ShowViews(UnresolvedNamespace(Seq.empty[String]), None)) + comparePlans( + parsePlan("SHOW VIEWS '*test*'"), + ShowViews(UnresolvedNamespace(Seq.empty[String]), Some("*test*"))) + comparePlans( + parsePlan("SHOW VIEWS LIKE '*test*'"), + ShowViews(UnresolvedNamespace(Seq.empty[String]), Some("*test*"))) + comparePlans( + parsePlan("SHOW VIEWS FROM testcat.ns1.ns2.tbl"), + ShowViews(UnresolvedNamespace(Seq("testcat", "ns1", "ns2", "tbl")), None)) + comparePlans( + parsePlan("SHOW VIEWS IN testcat.ns1.ns2.tbl"), + ShowViews(UnresolvedNamespace(Seq("testcat", "ns1", "ns2", "tbl")), None)) + comparePlans( + parsePlan("SHOW VIEWS IN ns1 '*test*'"), + ShowViews(UnresolvedNamespace(Seq("ns1")), Some("*test*"))) + comparePlans( + parsePlan("SHOW VIEWS IN ns1 LIKE '*test*'"), + ShowViews(UnresolvedNamespace(Seq("ns1")), Some("*test*"))) + } + test("create namespace -- backward compatibility with DATABASE/DBPROPERTIES") { val expected = CreateNamespaceStatement( Seq("a", "b", "c"), @@ -1525,7 +1559,7 @@ class DDLParserSuite extends AnalysisTest { AnalyzeColumnStatement(Seq("a", "b", "c"), None, allColumns = true)) intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value", - "mismatched input 'key' expecting ") + "mismatched input 'key' expecting {, ';'}") intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL", "missing 'COLUMNS' at ''") } @@ -2163,18 +2197,20 @@ class DDLParserSuite extends AnalysisTest { } test("create table - without using") { - val sql = "CREATE TABLE 1m.2g(a INT)" - val expectedTableSpec = TableSpec( - Seq("1m", "2g"), - Some(new StructType().add("a", IntegerType)), - Seq.empty[Transform], - None, - Map.empty[String, String], - None, - Map.empty[String, String], - None, - None) + withSQLConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED.key -> "false") { + val sql = "CREATE TABLE 1m.2g(a INT)" + val expectedTableSpec = TableSpec( + Seq("1m", "2g"), + Some(new StructType().add("a", IntegerType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + None, + Map.empty[String, String], + None, + None) - testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 74fd48d922a3a..522d49e448f1f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -772,10 +772,10 @@ class ExpressionParserSuite extends AnalysisTest { } test("timestamp literals") { - DateTimeTestUtils.outstandingTimezones.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone.getID) { + DateTimeTestUtils.outstandingZoneIds.foreach { zid => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId) { def toMicros(time: LocalDateTime): Long = { - val seconds = time.atZone(timeZone.toZoneId).toInstant.getEpochSecond + val seconds = time.atZone(zid).toInstant.getEpochSecond TimeUnit.SECONDS.toMicros(seconds) } assertEval( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 252a05a2d0c91..baa718bce3a50 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -1036,4 +1036,11 @@ class PlanParserSuite extends AnalysisTest { "t" -> ((table("a").select(Literal(0).as("level"), 'c) .union(table("t").where('level < 10).select('level + 1, 'c)), Seq("x"))))) } + + test("statement containing terminal semicolons") { + assertEqual("select 1;", OneRowRelation().select(1)) + assertEqual("select a, b;", OneRowRelation().select('a, 'b)) + assertEqual("select a, b from db.c;;;", table("db", "c").select('a, 'b)) + assertEqual("select a, b from db.c; ;; ;", table("db", "c").select('a, 'b)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 067c13b42f173..2db798b7714f9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -254,6 +254,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "utctimestamp", "values", "view", + "views", "while", "with", "work", @@ -526,6 +527,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "using", "values", "view", + "views", "when", "where", "window", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 912322a3af79d..8feb70ddc30e7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -109,10 +109,17 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => case sample: Sample => sample.copy(seed = 0L) case Join(left, right, joinType, condition, hint) if condition.isDefined => + val newJoinType = joinType match { + case ExistenceJoin(a: Attribute) => + val newAttr = AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) + ExistenceJoin(newAttr) + case other => other + } + val newCondition = splitConjunctivePredicates(condition.get).map(rewriteEqual).sortBy(_.hashCode()) .reduce(And) - Join(left, right, joinType, Some(newCondition), hint) + Join(left, right, newJoinType, Some(newCondition), hint) case w @ With(_, cteRelations, _) => w.copy(cteRelations = cteRelations.map { case (cteName, ctePlan) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala index d96f8086a3e93..91ce187f4d270 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala @@ -78,7 +78,7 @@ class QueryPlanSuite extends SparkFunSuite { val countRelationsInPlan = plan.collect({ case _: UnresolvedRelation => 1 }).sum val countRelationsInPlanAndSubqueries = - plan.collectInPlanAndSubqueries({ case _: UnresolvedRelation => 1 }).sum + plan.collectWithSubqueries({ case _: UnresolvedRelation => 1 }).sum assert(countRelationsInPlan == 2) assert(countRelationsInPlanAndSubqueries == 5) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index e72b2e9b1b214..f5259706325eb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.{AliasIdentifier, FunctionIdentifier, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.dsl.expressions.DslString +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.plans.{LeftOuter, NaturalJoin, SQLHelper} @@ -594,7 +595,8 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { } test("toJSON should not throws java.lang.StackOverflowError") { - val udf = ScalaUDF(SelfReferenceUDF(), BooleanType, Seq("col1".attr), false :: Nil) + val udf = ScalaUDF(SelfReferenceUDF(), BooleanType, Seq("col1".attr), + Option(ExpressionEncoder[String]()) :: Nil) // Should not throw java.lang.StackOverflowError udf.toJSON } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala new file mode 100644 index 0000000000000..817e503584324 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala @@ -0,0 +1,48 @@ +/* + * 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.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper._ + +class DateTimeFormatterHelperSuite extends SparkFunSuite { + + test("check incompatible pattern") { + assert(convertIncompatiblePattern("MM-DD-u") === "MM-DD-e") + assert(convertIncompatiblePattern("yyyy-MM-dd'T'HH:mm:ss.SSSz") + === "uuuu-MM-dd'T'HH:mm:ss.SSSz") + assert(convertIncompatiblePattern("yyyy-MM'y contains in quoted text'HH:mm:ss") + === "uuuu-MM'y contains in quoted text'HH:mm:ss") + assert(convertIncompatiblePattern("yyyy-MM-dd-u'T'HH:mm:ss.SSSz") + === "uuuu-MM-dd-e'T'HH:mm:ss.SSSz") + assert(convertIncompatiblePattern("yyyy-MM'u contains in quoted text'HH:mm:ss") + === "uuuu-MM'u contains in quoted text'HH:mm:ss") + assert(convertIncompatiblePattern("yyyy-MM'u contains in quoted text'''''HH:mm:ss") + === "uuuu-MM'u contains in quoted text'''''HH:mm:ss") + assert(convertIncompatiblePattern("yyyy-MM-dd'T'HH:mm:ss.SSSz G") + === "yyyy-MM-dd'T'HH:mm:ss.SSSz G") + unsupportedLetters.foreach { l => + val e = intercept[IllegalArgumentException](convertIncompatiblePattern(s"yyyy-MM-dd $l G")) + assert(e.getMessage === s"Illegal pattern character: $l") + } + assert(convertIncompatiblePattern("yyyy-MM-dd uuuu") === "uuuu-MM-dd eeee") + assert(convertIncompatiblePattern("yyyy-MM-dd EEEE") === "uuuu-MM-dd EEEE") + assert(convertIncompatiblePattern("yyyy-MM-dd'e'HH:mm:ss") === "uuuu-MM-dd'e'HH:mm:ss") + assert(convertIncompatiblePattern("yyyy-MM-dd'T'") === "uuuu-MM-dd'T'") + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala index 5f1428f96e2b0..bf9e8f71ba1c9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala @@ -21,34 +21,46 @@ import java.time.{LocalDate, LocalDateTime, LocalTime, ZoneId, ZoneOffset} import java.util.TimeZone import java.util.concurrent.TimeUnit +import scala.collection.JavaConverters._ + import org.apache.spark.sql.catalyst.util.DateTimeConstants._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.getZoneId /** * Helper functions for testing date and time functionality. */ object DateTimeTestUtils { - val ALL_TIMEZONES: Seq[TimeZone] = TimeZone.getAvailableIDs.toSeq.map(TimeZone.getTimeZone) + val CEST = getZoneId("+02:00") + val CET = getZoneId("+01:00") + val JST = getZoneId("+09:00") + val LA = getZoneId("America/Los_Angeles") + val MIT = getZoneId("-09:30") + val PST = getZoneId("-08:00") + val UTC = getZoneId("+00:00") + + val UTC_OPT = Option("UTC") + + val ALL_TIMEZONES: Seq[ZoneId] = ZoneId.getAvailableZoneIds.asScala.map(getZoneId).toSeq val outstandingTimezonesIds: Seq[String] = Seq( "UTC", - "PST", - "CET", + PST.getId, + CET.getId, "Africa/Dakar", - "America/Los_Angeles", + LA.getId, "Antarctica/Vostok", "Asia/Hong_Kong", "Europe/Amsterdam") - val outstandingTimezones: Seq[TimeZone] = outstandingTimezonesIds.map(TimeZone.getTimeZone) - val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(DateTimeUtils.getZoneId) + val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(getZoneId) - def withDefaultTimeZone[T](newDefaultTimeZone: TimeZone)(block: => T): T = { - val originalDefaultTimeZone = TimeZone.getDefault + def withDefaultTimeZone[T](newDefaultTimeZone: ZoneId)(block: => T): T = { + val originalDefaultTimeZone = ZoneId.systemDefault() try { - TimeZone.setDefault(newDefaultTimeZone) + TimeZone.setDefault(TimeZone.getTimeZone(newDefaultTimeZone)) block } finally { - TimeZone.setDefault(originalDefaultTimeZone) + TimeZone.setDefault(TimeZone.getTimeZone(originalDefaultTimeZone)) } } 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 68a4a24d6c9c7..2337a6a9e2254 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 @@ -19,8 +19,8 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneId, ZoneOffset} -import java.util.{Locale, TimeZone} +import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneId} +import java.util.Locale import java.util.concurrent.TimeUnit import org.scalatest.Matchers @@ -34,9 +34,6 @@ import org.apache.spark.unsafe.types.UTF8String class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { - val zonePST = getZoneId("PST") - val zoneGMT = getZoneId("GMT") - private def defaultZoneId = ZoneId.systemDefault() test("nanoseconds truncation") { @@ -88,13 +85,9 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } test("SPARK-6785: java date conversion before and after epoch") { - def format(d: Date): String = { - TimestampFormatter("yyyy-MM-dd", defaultTimeZone().toZoneId) - .format(millisToMicros(d.getTime)) - } def checkFromToJavaDate(d1: Date): Unit = { val d2 = toJavaDate(fromJavaDate(d1)) - assert(format(d2) === format(d1)) + assert(d2.toString === d1.toString) } val df1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) @@ -128,7 +121,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { checkFromToJavaDate(new Date(df2.parse("1776-07-04 18:30:00 UTC").getTime)) } - private def toDate(s: String, zoneId: ZoneId = ZoneOffset.UTC): Option[SQLDate] = { + private def toDate(s: String, zoneId: ZoneId = UTC): Option[SQLDate] = { stringToDate(UTF8String.fromString(s), zoneId) } @@ -161,11 +154,10 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } test("string to timestamp") { - for (tz <- ALL_TIMEZONES) { + for (zid <- ALL_TIMEZONES) { def checkStringToTimestamp(str: String, expected: Option[Long]): Unit = { - assert(toTimestamp(str, tz.toZoneId) === expected) + assert(toTimestamp(str, zid) === expected) } - val zid = tz.toZoneId checkStringToTimestamp("1969-12-31 16:00:00", Option(date(1969, 12, 31, 16, zid = zid))) checkStringToTimestamp("0001", Option(date(1, 1, 1, 0, zid = zid))) @@ -180,29 +172,28 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { // If the string value includes timezone string, it represents the timestamp string // in the timezone regardless of the tz parameter. - var zoneId = getZoneId("GMT-13:53") + var zoneId = getZoneId("-13:53") expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId)) checkStringToTimestamp("2015-03-18T12:03:17-13:53", expected) checkStringToTimestamp("2015-03-18T12:03:17GMT-13:53", expected) - zoneId = getZoneId("UTC") - expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId)) + expected = Option(date(2015, 3, 18, 12, 3, 17, zid = UTC)) checkStringToTimestamp("2015-03-18T12:03:17Z", expected) checkStringToTimestamp("2015-03-18 12:03:17Z", expected) checkStringToTimestamp("2015-03-18 12:03:17UTC", expected) - zoneId = getZoneId("GMT-01:00") + zoneId = getZoneId("-01:00") expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId)) checkStringToTimestamp("2015-03-18T12:03:17-1:0", expected) checkStringToTimestamp("2015-03-18T12:03:17-01:00", expected) checkStringToTimestamp("2015-03-18T12:03:17GMT-01:00", expected) - zoneId = getZoneId("GMT+07:30") + zoneId = getZoneId("+07:30") expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId)) checkStringToTimestamp("2015-03-18T12:03:17+07:30", expected) checkStringToTimestamp("2015-03-18T12:03:17 GMT+07:30", expected) - zoneId = getZoneId("GMT+07:03") + zoneId = getZoneId("+07:03") expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId)) checkStringToTimestamp("2015-03-18T12:03:17+07:03", expected) checkStringToTimestamp("2015-03-18T12:03:17GMT+07:03", expected) @@ -214,24 +205,23 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { // If the string value includes timezone string, it represents the timestamp string // in the timezone regardless of the tz parameter. - zoneId = getZoneId("UTC") - expected = Option(date(2015, 3, 18, 12, 3, 17, 456000, zid = zoneId)) + expected = Option(date(2015, 3, 18, 12, 3, 17, 456000, zid = UTC)) checkStringToTimestamp("2015-03-18T12:03:17.456Z", expected) checkStringToTimestamp("2015-03-18 12:03:17.456Z", expected) checkStringToTimestamp("2015-03-18 12:03:17.456 UTC", expected) - zoneId = getZoneId("GMT-01:00") + zoneId = getZoneId("-01:00") expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, zid = zoneId)) checkStringToTimestamp("2015-03-18T12:03:17.123-1:0", expected) checkStringToTimestamp("2015-03-18T12:03:17.123-01:00", expected) checkStringToTimestamp("2015-03-18T12:03:17.123 GMT-01:00", expected) - zoneId = getZoneId("GMT+07:30") + zoneId = getZoneId("+07:30") expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, zid = zoneId)) checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", expected) checkStringToTimestamp("2015-03-18T12:03:17.123 GMT+07:30", expected) - zoneId = getZoneId("GMT+07:30") + zoneId = getZoneId("+07:30") expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, zid = zoneId)) checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", expected) checkStringToTimestamp("2015-03-18T12:03:17.123GMT+07:30", expected) @@ -240,7 +230,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { checkStringToTimestamp("2015-03-18T12:03:17.123121+7:30", expected) checkStringToTimestamp("2015-03-18T12:03:17.123121 GMT+0730", expected) - zoneId = getZoneId("GMT+07:30") + zoneId = getZoneId("+07:30") expected = Option(date(2015, 3, 18, 12, 3, 17, 123120, zid = zoneId)) checkStringToTimestamp("2015-03-18T12:03:17.12312+7:30", expected) checkStringToTimestamp("2015-03-18T12:03:17.12312 UT+07:30", expected) @@ -248,12 +238,12 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { expected = Option(time(18, 12, 15, zid = zid)) checkStringToTimestamp("18:12:15", expected) - zoneId = getZoneId("GMT+07:30") + zoneId = getZoneId("+07:30") expected = Option(time(18, 12, 15, 123120, zid = zoneId)) checkStringToTimestamp("T18:12:15.12312+7:30", expected) checkStringToTimestamp("T18:12:15.12312 UTC+07:30", expected) - zoneId = getZoneId("GMT+07:30") + zoneId = getZoneId("+07:30") expected = Option(time(18, 12, 15, 123120, zid = zoneId)) checkStringToTimestamp("18:12:15.12312+7:30", expected) checkStringToTimestamp("18:12:15.12312 GMT+07:30", expected) @@ -280,8 +270,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { checkStringToTimestamp("1999 08", None) // Truncating the fractional seconds - zoneId = getZoneId("GMT+00:00") - expected = Option(date(2015, 3, 18, 12, 3, 17, 123456, zid = zoneId)) + expected = Option(date(2015, 3, 18, 12, 3, 17, 123456, zid = UTC)) checkStringToTimestamp("2015-03-18T12:03:17.123456789+0:00", expected) checkStringToTimestamp("2015-03-18T12:03:17.123456789 UTC+0", expected) checkStringToTimestamp("2015-03-18T12:03:17.123456789GMT+00:00", expected) @@ -310,38 +299,38 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } test("hours") { - var input = date(2015, 3, 18, 13, 2, 11, 0, zonePST) - assert(getHours(input, zonePST) === 13) - assert(getHours(input, zoneGMT) === 20) - input = date(2015, 12, 8, 2, 7, 9, 0, zonePST) - assert(getHours(input, zonePST) === 2) - assert(getHours(input, zoneGMT) === 10) - input = date(10, 1, 1, 0, 0, 0, 0, zonePST) - assert(getHours(input, zonePST) === 0) + var input = date(2015, 3, 18, 13, 2, 11, 0, LA) + assert(getHours(input, LA) === 13) + assert(getHours(input, UTC) === 20) + input = date(2015, 12, 8, 2, 7, 9, 0, LA) + assert(getHours(input, LA) === 2) + assert(getHours(input, UTC) === 10) + input = date(10, 1, 1, 0, 0, 0, 0, LA) + assert(getHours(input, LA) === 0) } test("minutes") { - var input = date(2015, 3, 18, 13, 2, 11, 0, zonePST) - assert(getMinutes(input, zonePST) === 2) - assert(getMinutes(input, zoneGMT) === 2) + var input = date(2015, 3, 18, 13, 2, 11, 0, LA) + assert(getMinutes(input, LA) === 2) + assert(getMinutes(input, UTC) === 2) assert(getMinutes(input, getZoneId("Australia/North")) === 32) - input = date(2015, 3, 8, 2, 7, 9, 0, zonePST) - assert(getMinutes(input, zonePST) === 7) - assert(getMinutes(input, zoneGMT) === 7) + input = date(2015, 3, 8, 2, 7, 9, 0, LA) + assert(getMinutes(input, LA) === 7) + assert(getMinutes(input, UTC) === 7) assert(getMinutes(input, getZoneId("Australia/North")) === 37) - input = date(10, 1, 1, 0, 0, 0, 0, zonePST) - assert(getMinutes(input, zonePST) === 0) + input = date(10, 1, 1, 0, 0, 0, 0, LA) + assert(getMinutes(input, LA) === 0) } test("seconds") { - var input = date(2015, 3, 18, 13, 2, 11, 0, zonePST) - assert(getSeconds(input, zonePST) === 11) - assert(getSeconds(input, zoneGMT) === 11) - input = date(2015, 3, 8, 2, 7, 9, 0, zonePST) - assert(getSeconds(input, zonePST) === 9) - assert(getSeconds(input, zoneGMT) === 9) - input = date(10, 1, 1, 0, 0, 0, 0, zonePST) - assert(getSeconds(input, zonePST) === 0) + var input = date(2015, 3, 18, 13, 2, 11, 0, LA) + assert(getSeconds(input, LA) === 11) + assert(getSeconds(input, UTC) === 11) + input = date(2015, 3, 8, 2, 7, 9, 0, LA) + assert(getSeconds(input, LA) === 9) + assert(getSeconds(input, UTC) === 9) + input = date(10, 1, 1, 0, 0, 0, 0, LA) + assert(getSeconds(input, LA) === 0) } test("hours / minutes / seconds") { @@ -407,58 +396,58 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { val ts2 = date(2000, 2, 28, 10, 30, 0, 123000) assert(timestampAddInterval(ts1, 36, 0, 123000, defaultZoneId) === ts2) - val ts3 = date(1997, 2, 27, 16, 0, 0, 0, zonePST) - val ts4 = date(2000, 2, 27, 16, 0, 0, 123000, zonePST) - val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, zoneGMT) - assert(timestampAddInterval(ts3, 36, 0, 123000, zonePST) === ts4) - assert(timestampAddInterval(ts3, 36, 0, 123000, zoneGMT) === ts5) + val ts3 = date(1997, 2, 27, 16, 0, 0, 0, LA) + val ts4 = date(2000, 2, 27, 16, 0, 0, 123000, LA) + val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, UTC) + assert(timestampAddInterval(ts3, 36, 0, 123000, LA) === ts4) + assert(timestampAddInterval(ts3, 36, 0, 123000, UTC) === ts5) } test("timestamp add days") { // 2019-3-9 is the end of Pacific Standard Time - val ts1 = date(2019, 3, 9, 12, 0, 0, 123000, zonePST) + val ts1 = date(2019, 3, 9, 12, 0, 0, 123000, LA) // 2019-3-10 is the start of Pacific Daylight Time - val ts2 = date(2019, 3, 10, 12, 0, 0, 123000, zonePST) - val ts3 = date(2019, 5, 9, 12, 0, 0, 123000, zonePST) - val ts4 = date(2019, 5, 10, 12, 0, 0, 123000, zonePST) + val ts2 = date(2019, 3, 10, 12, 0, 0, 123000, LA) + val ts3 = date(2019, 5, 9, 12, 0, 0, 123000, LA) + val ts4 = date(2019, 5, 10, 12, 0, 0, 123000, LA) // 2019-11-2 is the end of Pacific Daylight Time - val ts5 = date(2019, 11, 2, 12, 0, 0, 123000, zonePST) + val ts5 = date(2019, 11, 2, 12, 0, 0, 123000, LA) // 2019-11-3 is the start of Pacific Standard Time - val ts6 = date(2019, 11, 3, 12, 0, 0, 123000, zonePST) + val ts6 = date(2019, 11, 3, 12, 0, 0, 123000, LA) // transit from Pacific Standard Time to Pacific Daylight Time assert(timestampAddInterval( - ts1, 0, 0, 23 * MICROS_PER_HOUR, zonePST) === ts2) - assert(timestampAddInterval(ts1, 0, 1, 0, zonePST) === ts2) + ts1, 0, 0, 23 * MICROS_PER_HOUR, LA) === ts2) + assert(timestampAddInterval(ts1, 0, 1, 0, LA) === ts2) // just a normal day assert(timestampAddInterval( - ts3, 0, 0, 24 * MICROS_PER_HOUR, zonePST) === ts4) - assert(timestampAddInterval(ts3, 0, 1, 0, zonePST) === ts4) + ts3, 0, 0, 24 * MICROS_PER_HOUR, LA) === ts4) + assert(timestampAddInterval(ts3, 0, 1, 0, LA) === ts4) // transit from Pacific Daylight Time to Pacific Standard Time assert(timestampAddInterval( - ts5, 0, 0, 25 * MICROS_PER_HOUR, zonePST) === ts6) - assert(timestampAddInterval(ts5, 0, 1, 0, zonePST) === ts6) + ts5, 0, 0, 25 * MICROS_PER_HOUR, LA) === ts6) + assert(timestampAddInterval(ts5, 0, 1, 0, LA) === ts6) } test("monthsBetween") { val date1 = date(1997, 2, 28, 10, 30, 0) var date2 = date(1996, 10, 30) - assert(monthsBetween(date1, date2, true, ZoneOffset.UTC) === 3.94959677) - assert(monthsBetween(date1, date2, false, ZoneOffset.UTC) === 3.9495967741935485) + assert(monthsBetween(date1, date2, true, UTC) === 3.94959677) + assert(monthsBetween(date1, date2, false, UTC) === 3.9495967741935485) Seq(true, false).foreach { roundOff => date2 = date(2000, 2, 28) - assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36) + assert(monthsBetween(date1, date2, roundOff, UTC) === -36) date2 = date(2000, 2, 29) - assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36) + assert(monthsBetween(date1, date2, roundOff, UTC) === -36) date2 = date(1996, 3, 31) - assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === 11) + assert(monthsBetween(date1, date2, roundOff, UTC) === 11) } - val date3 = date(2000, 2, 28, 16, zid = zonePST) - val date4 = date(1997, 2, 28, 16, zid = zonePST) - assert(monthsBetween(date3, date4, true, zonePST) === 36.0) - assert(monthsBetween(date3, date4, true, ZoneOffset.UTC) === 35.90322581) - assert(monthsBetween(date3, date4, false, ZoneOffset.UTC) === 35.903225806451616) + val date3 = date(2000, 2, 28, 16, zid = LA) + val date4 = date(1997, 2, 28, 16, zid = LA) + assert(monthsBetween(date3, date4, true, LA) === 36.0) + assert(monthsBetween(date3, date4, true, UTC) === 35.90322581) + assert(monthsBetween(date3, date4, false, UTC) === 35.903225806451616) } test("from UTC timestamp") { @@ -469,19 +458,19 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { for (tz <- ALL_TIMEZONES) { withDefaultTimeZone(tz) { test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456") - test("2011-12-25 09:00:00.123456", "JST", "2011-12-25 18:00:00.123456") - test("2011-12-25 09:00:00.123456", "PST", "2011-12-25 01:00:00.123456") + test("2011-12-25 09:00:00.123456", JST.getId, "2011-12-25 18:00:00.123456") + test("2011-12-25 09:00:00.123456", LA.getId, "2011-12-25 01:00:00.123456") test("2011-12-25 09:00:00.123456", "Asia/Shanghai", "2011-12-25 17:00:00.123456") } } - withDefaultTimeZone(TimeZone.getTimeZone("PST")) { + withDefaultTimeZone(LA) { // Daylight Saving Time - test("2016-03-13 09:59:59.0", "PST", "2016-03-13 01:59:59.0") - test("2016-03-13 10:00:00.0", "PST", "2016-03-13 03:00:00.0") - test("2016-11-06 08:59:59.0", "PST", "2016-11-06 01:59:59.0") - test("2016-11-06 09:00:00.0", "PST", "2016-11-06 01:00:00.0") - test("2016-11-06 10:00:00.0", "PST", "2016-11-06 02:00:00.0") + test("2016-03-13 09:59:59.0", LA.getId, "2016-03-13 01:59:59.0") + test("2016-03-13 10:00:00.0", LA.getId, "2016-03-13 03:00:00.0") + test("2016-11-06 08:59:59.0", LA.getId, "2016-11-06 01:59:59.0") + test("2016-11-06 09:00:00.0", LA.getId, "2016-11-06 01:00:00.0") + test("2016-11-06 10:00:00.0", LA.getId, "2016-11-06 02:00:00.0") } } @@ -491,26 +480,25 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { === expected) } - for (tz <- ALL_TIMEZONES) { - withDefaultTimeZone(tz) { + for (zid <- ALL_TIMEZONES) { + withDefaultTimeZone(zid) { test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456") - test("2011-12-25 18:00:00.123456", "JST", "2011-12-25 09:00:00.123456") - test("2011-12-25 01:00:00.123456", "PST", "2011-12-25 09:00:00.123456") + test("2011-12-25 18:00:00.123456", JST.getId, "2011-12-25 09:00:00.123456") + test("2011-12-25 01:00:00.123456", LA.getId, "2011-12-25 09:00:00.123456") test("2011-12-25 17:00:00.123456", "Asia/Shanghai", "2011-12-25 09:00:00.123456") } } - withDefaultTimeZone(TimeZone.getTimeZone("PST")) { + withDefaultTimeZone(LA) { + val tz = LA.getId // Daylight Saving Time - test("2016-03-13 01:59:59", "PST", "2016-03-13 09:59:59.0") - // 2016-03-13 02:00:00 PST does not exists - test("2016-03-13 02:00:00", "PST", "2016-03-13 10:00:00.0") - test("2016-03-13 03:00:00", "PST", "2016-03-13 10:00:00.0") - test("2016-11-06 00:59:59", "PST", "2016-11-06 07:59:59.0") - // 2016-11-06 01:00:00 PST could be 2016-11-06 08:00:00 UTC or 2016-11-06 09:00:00 UTC - test("2016-11-06 01:00:00", "PST", "2016-11-06 09:00:00.0") - test("2016-11-06 01:59:59", "PST", "2016-11-06 09:59:59.0") - test("2016-11-06 02:00:00", "PST", "2016-11-06 10:00:00.0") + test("2016-03-13 01:59:59", tz, "2016-03-13 09:59:59.0") + test("2016-03-13 02:00:00", tz, "2016-03-13 10:00:00.0") + test("2016-03-13 03:00:00", tz, "2016-03-13 10:00:00.0") + test("2016-11-06 00:59:59", tz, "2016-11-06 07:59:59.0") + test("2016-11-06 01:00:00", tz, "2016-11-06 09:00:00.0") + test("2016-11-06 01:59:59", tz, "2016-11-06 09:59:59.0") + test("2016-11-06 02:00:00", tz, "2016-11-06 10:00:00.0") } } @@ -563,9 +551,8 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { testTrunc(DateTimeUtils.TRUNC_TO_CENTURY, "2001-01-01", defaultInputTS.get) testTrunc(DateTimeUtils.TRUNC_TO_MILLENNIUM, "2001-01-01", defaultInputTS.get) - for (tz <- ALL_TIMEZONES) { - withDefaultTimeZone(tz) { - val zid = tz.toZoneId + for (zid <- ALL_TIMEZONES) { + withDefaultTimeZone(zid) { val inputTS = DateTimeUtils.stringToTimestamp( UTF8String.fromString("2015-03-05T09:32:05.359"), defaultZoneId) val inputTS1 = DateTimeUtils.stringToTimestamp( @@ -601,16 +588,16 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } test("daysToMicros and microsToDays") { - val input = date(2015, 12, 31, 16, zid = zonePST) - assert(microsToDays(input, zonePST) === 16800) - assert(microsToDays(input, ZoneOffset.UTC) === 16801) - assert(microsToDays(-1 * MILLIS_PER_DAY + 1, ZoneOffset.UTC) == -1) + val input = date(2015, 12, 31, 16, zid = LA) + assert(microsToDays(input, LA) === 16800) + assert(microsToDays(input, UTC) === 16801) + assert(microsToDays(-1 * MILLIS_PER_DAY + 1, UTC) == -1) - var expected = date(2015, 12, 31, zid = zonePST) - assert(daysToMicros(16800, zonePST) === expected) + var expected = date(2015, 12, 31, zid = LA) + assert(daysToMicros(16800, LA) === expected) - expected = date(2015, 12, 31, zid = zoneGMT) - assert(daysToMicros(16800, ZoneOffset.UTC) === expected) + expected = date(2015, 12, 31, zid = UTC) + assert(daysToMicros(16800, UTC) === expected) // There are some days are skipped entirely in some timezone, skip them here. val skipped_days = Map[String, Set[Int]]( @@ -620,15 +607,15 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { "Pacific/Fakaofo" -> Set(15338), "Pacific/Kiritimati" -> Set(9130, 9131), "Pacific/Kwajalein" -> Set(8632, 8633, 8634), - "MIT" -> Set(15338)) - for (tz <- ALL_TIMEZONES) { - val skipped = skipped_days.getOrElse(tz.getID, Set.empty) + MIT.getId -> Set(15338)) + for (zid <- ALL_TIMEZONES) { + val skipped = skipped_days.getOrElse(zid.getId, Set.empty) val testingData = Seq(-20000, 20000) ++ (1 to 1000).map(_ => (math.random() * 40000 - 20000).toInt) testingData.foreach { d => if (!skipped.contains(d)) { - assert(microsToDays(daysToMicros(d, tz.toZoneId), tz.toZoneId) === d, - s"Round trip of $d did not work in tz $tz") + assert(microsToDays(daysToMicros(d, zid), zid) === d, + s"Round trip of $d did not work in tz ${zid.getId}") } } } @@ -670,20 +657,4 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { assert(toDate("tomorrow CET ", zoneId).get === today + 1) } } - - test("check incompatible pattern") { - assert(convertIncompatiblePattern("MM-DD-u") === "MM-DD-e") - assert(convertIncompatiblePattern("yyyy-MM-dd'T'HH:mm:ss.SSSz") - === "uuuu-MM-dd'T'HH:mm:ss.SSSz") - assert(convertIncompatiblePattern("yyyy-MM'y contains in quoted text'HH:mm:ss") - === "uuuu-MM'y contains in quoted text'HH:mm:ss") - assert(convertIncompatiblePattern("yyyy-MM-dd-u'T'HH:mm:ss.SSSz") - === "uuuu-MM-dd-e'T'HH:mm:ss.SSSz") - assert(convertIncompatiblePattern("yyyy-MM'u contains in quoted text'HH:mm:ss") - === "uuuu-MM'u contains in quoted text'HH:mm:ss") - assert(convertIncompatiblePattern("yyyy-MM'u contains in quoted text'''''HH:mm:ss") - === "uuuu-MM'u contains in quoted text'''''HH:mm:ss") - assert(convertIncompatiblePattern("yyyy-MM-dd'T'HH:mm:ss.SSSz G") - === "yyyy-MM-dd'T'HH:mm:ss.SSSz G") - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala new file mode 100644 index 0000000000000..15fd90c6798b0 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -0,0 +1,367 @@ +/* + * 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.util + +import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate, LocalDateTime, ZoneId} +import java.util.TimeZone + +import org.scalatest.Matchers + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.DateTimeConstants._ +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.catalyst.util.RebaseDateTime._ + +class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { + + private def toJulianMicros(ts: Timestamp): Long = { + val julianMicros = millisToMicros(ts.getTime) + + ((ts.getNanos / NANOS_PER_MICROS) % MICROS_PER_MILLIS) + julianMicros + } + private def parseToJulianMicros(s: String): Long = toJulianMicros(Timestamp.valueOf(s)) + + private def toGregorianMicros(ldt: LocalDateTime, zoneId: ZoneId): Long = { + instantToMicros(ldt.atZone(zoneId).toInstant) + } + private def parseToGregMicros(s: String, zoneId: ZoneId): Long = { + toGregorianMicros(LocalDateTime.parse(s), zoneId) + } + + test("rebase julian to/from gregorian micros") { + outstandingZoneIds.foreach { zid => + withDefaultTimeZone(zid) { + Seq( + "0001-01-01 01:02:03.654321", + "1000-01-01 03:02:01.123456", + "1582-10-04 00:00:00.000000", + "1582-10-15 00:00:00.999999", // Gregorian cutover day + "1883-11-10 00:00:00.000000", // America/Los_Angeles -7:52:58 zone offset + "1883-11-20 00:00:00.000000", // America/Los_Angeles -08:00 zone offset + "1969-12-31 11:22:33.000100", + "1970-01-01 00:00:00.000001", // The epoch day + "2020-03-14 09:33:01.500000").foreach { ts => + withClue(s"time zone = ${zid.getId} ts = $ts") { + val julianMicros = parseToJulianMicros(ts) + val gregMicros = parseToGregMicros(ts.replace(' ', 'T'), zid) + + assert(rebaseJulianToGregorianMicros(julianMicros) === gregMicros) + assert(rebaseGregorianToJulianMicros(gregMicros) === julianMicros) + } + } + } + } + } + + // millisToDays() and fromJavaDate() are taken from Spark 2.4 + private def millisToDaysLegacy(millisUtc: Long, timeZone: TimeZone): Int = { + val millisLocal = millisUtc + timeZone.getOffset(millisUtc) + Math.floor(millisLocal.toDouble / MILLIS_PER_DAY).toInt + } + private def fromJavaDateLegacy(date: Date): Int = { + millisToDaysLegacy(date.getTime, defaultTimeZone()) + } + + test("rebase gregorian to/from julian days") { + outstandingZoneIds.foreach { zid => + withDefaultTimeZone(zid) { + Seq( + "0001-01-01", + "1000-01-01", + "1582-10-04", + "1582-10-15", // Gregorian cutover day + "1883-11-10", // America/Los_Angeles -7:52:58 zone offset + "1883-11-20", // America/Los_Angeles -08:00 zone offset + "1969-12-31", + "1970-01-01", // The epoch day + "2020-03-14").foreach { date => + val julianDays = fromJavaDateLegacy(Date.valueOf(date)) + val gregorianDays = localDateToDays(LocalDate.parse(date)) + + assert(rebaseGregorianToJulianDays(gregorianDays) === julianDays) + assert(rebaseJulianToGregorianDays(julianDays) === gregorianDays) + } + } + } + } + + test("rebase julian to gregorian date for leap years") { + outstandingZoneIds.foreach { zid => + withDefaultTimeZone(zid) { + Seq( + "1000-02-29" -> "1000-03-01", + "1600-02-29" -> "1600-02-29", + "1700-02-29" -> "1700-03-01", + "2000-02-29" -> "2000-02-29").foreach { case (julianDate, gregDate) => + withClue(s"tz = ${zid.getId} julian date = $julianDate greg date = $gregDate") { + val date = Date.valueOf(julianDate) + val julianDays = fromJavaDateLegacy(date) + val gregorianDays = localDateToDays(LocalDate.parse(gregDate)) + + assert(rebaseJulianToGregorianDays(julianDays) === gregorianDays) + } + } + } + } + } + + test("rebase julian to gregorian timestamp for leap years") { + outstandingZoneIds.foreach { zid => + withDefaultTimeZone(zid) { + Seq( + "1000-02-29 01:02:03.123456" -> "1000-03-01T01:02:03.123456", + "1600-02-29 11:12:13.654321" -> "1600-02-29T11:12:13.654321", + "1700-02-29 21:22:23.000001" -> "1700-03-01T21:22:23.000001", + "2000-02-29 00:00:00.999999" -> "2000-02-29T00:00:00.999999" + ).foreach { case (julianTs, gregTs) => + withClue(s"tz = ${zid.getId} julian ts = $julianTs greg ts = $gregTs") { + val julianMicros = parseToJulianMicros(julianTs) + val gregorianMicros = parseToGregMicros(gregTs, zid) + + assert(rebaseJulianToGregorianMicros(julianMicros) === gregorianMicros) + } + } + } + } + } + + test("optimization of days rebasing - Gregorian to Julian") { + val start = localDateToDays(LocalDate.of(1, 1, 1)) + val end = localDateToDays(LocalDate.of(2030, 1, 1)) + + var days = start + while (days < end) { + assert(rebaseGregorianToJulianDays(days) === localRebaseGregorianToJulianDays(days)) + days += 1 + } + } + + test("optimization of days rebasing - Julian to Gregorian") { + val start = rebaseGregorianToJulianDays( + localDateToDays(LocalDate.of(1, 1, 1))) + val end = rebaseGregorianToJulianDays( + localDateToDays(LocalDate.of(2030, 1, 1))) + + var days = start + while (days < end) { + assert(rebaseJulianToGregorianDays(days) === localRebaseJulianToGregorianDays(days)) + days += 1 + } + } + + test("SPARK-31328: rebasing overlapped timestamps during daylight saving time") { + Seq( + LA.getId -> Seq("2019-11-03T08:00:00Z", "2019-11-03T08:30:00Z", "2019-11-03T09:00:00Z"), + "Europe/Amsterdam" -> + Seq("2019-10-27T00:00:00Z", "2019-10-27T00:30:00Z", "2019-10-27T01:00:00Z") + ).foreach { case (tz, ts) => + withDefaultTimeZone(getZoneId(tz)) { + ts.foreach { str => + val micros = instantToMicros(Instant.parse(str)) + assert(rebaseGregorianToJulianMicros(micros) === micros) + assert(rebaseJulianToGregorianMicros(micros) === micros) + } + } + } + } + + test("validate rebase records in JSON files") { + Seq( + "gregorian-julian-rebase-micros.json", + "julian-gregorian-rebase-micros.json").foreach { json => + withClue(s"JSON file = $json") { + val rebaseRecords = loadRebaseRecords(json) + rebaseRecords.foreach { case (_, rebaseRecord) => + assert(rebaseRecord.switches.size === rebaseRecord.diffs.size) + // Check ascending order of switches values + assert(rebaseRecord.switches.toSeq === rebaseRecord.switches.sorted.toSeq) + } + } + } + } + + test("optimization of micros rebasing - Gregorian to Julian") { + outstandingZoneIds.foreach { zid => + withClue(s"zone id = $zid") { + withDefaultTimeZone(zid) { + val start = instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0) + .atZone(zid) + .toInstant) + val end = instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0) + .atZone(zid) + .toInstant) + var micros = start + do { + val rebased = rebaseGregorianToJulianMicros(zid, micros) + val rebasedAndOptimized = rebaseGregorianToJulianMicros(micros) + assert(rebasedAndOptimized === rebased) + micros += (MICROS_PER_MONTH * (0.5 + Math.random())).toLong + } while (micros <= end) + } + } + } + } + + test("optimization of micros rebasing - Julian to Gregorian") { + outstandingZoneIds.foreach { zid => + withClue(s"zone id = $zid") { + withDefaultTimeZone(zid) { + val start = rebaseGregorianToJulianMicros( + instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0).atZone(zid).toInstant)) + val end = rebaseGregorianToJulianMicros( + instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0).atZone(zid).toInstant)) + var micros = start + do { + val rebased = rebaseJulianToGregorianMicros(zid, micros) + val rebasedAndOptimized = rebaseJulianToGregorianMicros(micros) + assert(rebasedAndOptimized === rebased) + micros += (MICROS_PER_MONTH * (0.5 + Math.random())).toLong + } while (micros <= end) + } + } + } + } + + private def generateRebaseJson( + adjustFunc: Long => Long, + rebaseFunc: (ZoneId, Long) => Long, + dir: String, + fileName: String): Unit = { + import java.nio.file.{Files, Paths} + import java.nio.file.StandardOpenOption + + import scala.collection.mutable.ArrayBuffer + + import com.fasterxml.jackson.databind.ObjectMapper + import com.fasterxml.jackson.module.scala.{DefaultScalaModule, ScalaObjectMapper} + + case class RebaseRecord(tz: String, switches: Array[Long], diffs: Array[Long]) + + val result = new ArrayBuffer[RebaseRecord]() + // The time zones are excluded because: + // 1. Julian to Gregorian rebasing doesn't match to the opposite rebasing from + // Gregorian to Julian rebasing. + // 2. Linear searching over switch points might be slow. + // 3. Results after the end time point 2100-01-01 are wrong. + // See SPARK-31385 + val blacklist = Set("Asia/Tehran", "Iran", "Africa/Casablanca", "Africa/El_Aaiun") + ALL_TIMEZONES + .filterNot(zid => blacklist.contains(zid.getId)) + .sortBy(_.getId) + .foreach { zid => + withDefaultTimeZone(zid) { + val start = adjustFunc(instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0) + .atZone(zid) + .toInstant)) + val end = adjustFunc(instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0) + .atZone(zid) + .toInstant)) + + var micros = start + var diff = Long.MaxValue + val maxStep = DAYS_PER_WEEK * MICROS_PER_DAY + var step: Long = MICROS_PER_SECOND + val switches = new ArrayBuffer[Long]() + val diffs = new ArrayBuffer[Long]() + while (micros < end) { + val rebased = rebaseFunc(zid, micros) + val curDiff = rebased - micros + if (curDiff != diff) { + if (step > MICROS_PER_SECOND) { + micros -= step + step = (Math.max(MICROS_PER_SECOND, step / 2) / MICROS_PER_SECOND) * MICROS_PER_SECOND + } else { + diff = curDiff + step = maxStep + assert(diff % MICROS_PER_SECOND == 0) + diffs.append(diff / MICROS_PER_SECOND) + assert(micros % MICROS_PER_SECOND == 0) + switches.append(micros / MICROS_PER_SECOND) + } + } + micros += step + } + result.append(RebaseRecord(zid.getId, switches.toArray, diffs.toArray)) + } + } + val mapper = (new ObjectMapper() with ScalaObjectMapper) + .registerModule(DefaultScalaModule) + .writerWithDefaultPrettyPrinter() + mapper.writeValue( + Files.newOutputStream( + Paths.get(dir, fileName), + StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.TRUNCATE_EXISTING), + result.toArray) + } + + ignore("generate 'gregorian-julian-rebase-micros.json'") { + generateRebaseJson( + adjustFunc = identity[Long], + rebaseFunc = rebaseGregorianToJulianMicros, + dir = "/Users/maximgekk/tmp", + fileName = "gregorian-julian-rebase-micros.json") + } + + ignore("generate 'julian-gregorian-rebase-micros.json'") { + generateRebaseJson( + adjustFunc = rebaseGregorianToJulianMicros, + rebaseFunc = rebaseJulianToGregorianMicros, + dir = "/Users/maximgekk/tmp", + fileName = "julian-gregorian-rebase-micros.json") + } + + test("rebase gregorian to/from julian days - BCE era") { + outstandingZoneIds.foreach { zid => + withDefaultTimeZone(zid) { + Seq( + (-1100, 1, 1), + (-1044, 3, 5), + (-44, 3, 5)).foreach { case (year, month, day) => + val julianDays = fromJavaDateLegacy(new Date(year - 1900, month - 1, day)) + val gregorianDays = localDateToDays(LocalDate.of(year, month, day)) + + assert(rebaseGregorianToJulianDays(gregorianDays) === julianDays) + assert(rebaseJulianToGregorianDays(julianDays) === gregorianDays) + } + } + } + } + + test("rebase gregorian to/from julian micros - BCE era") { + outstandingZoneIds.foreach { zid => + withDefaultTimeZone(zid) { + Seq( + (-1100, 1, 1, 1, 2, 3, 0), + (-1044, 3, 5, 0, 0, 0, 123456000), + (-44, 3, 5, 23, 59, 59, 99999000) + ).foreach { case (year, month, day, hour, minute, second, nanos) => + val julianMicros = toJulianMicros(new Timestamp( + year - 1900, month - 1, day, hour, minute, second, nanos)) + val gregorianMicros = toGregorianMicros(LocalDateTime.of( + year, month, day, hour, minute, second, nanos), zid) + + assert(rebaseGregorianToJulianMicros(gregorianMicros) === julianMicros) + assert(rebaseJulianToGregorianMicros(julianMicros) === gregorianMicros) + } + } + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index 0187ae31e2d1c..3d7026e180cd1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -116,6 +116,7 @@ class InMemoryTable( } override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + InMemoryTable.maybeSimulateFailedTableWrite(new CaseInsensitiveStringMap(properties)) InMemoryTable.maybeSimulateFailedTableWrite(info.options) new WriteBuilder with SupportsTruncate with SupportsOverwrite with SupportsDynamicOverwrite { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala index 4439a7bb3ae87..0ffb492e702d1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.util import org.apache.arrow.vector.types.pojo.ArrowType import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.LA import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ @@ -65,7 +66,7 @@ class ArrowUtilsSuite extends SparkFunSuite { roundtripWithTz(DateTimeUtils.defaultTimeZone().getID) roundtripWithTz("Asia/Tokyo") roundtripWithTz("UTC") - roundtripWithTz("America/Los_Angeles") + roundtripWithTz(LA.getId) } test("array") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala index d617b1c1d8237..a40dbcc2ec664 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.util -import java.time.{LocalDate, ZoneOffset} +import java.time.{DateTimeException, LocalDate, ZoneOffset} -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkUpgradeException} import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, localDateToDays} @@ -114,4 +114,26 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { assert(formatter.parse("tomorrow UTC") === today + 1) } } + + test("SPARK-30958: parse date with negative year") { + val formatter1 = DateFormatter("yyyy-MM-dd", ZoneOffset.UTC) + assert(formatter1.parse("-1234-02-22") === localDateToDays(LocalDate.of(-1234, 2, 22))) + + def assertParsingError(f: => Unit): Unit = { + intercept[Exception](f) match { + case e: SparkUpgradeException => + assert(e.getCause.isInstanceOf[DateTimeException]) + case e => + assert(e.isInstanceOf[DateTimeException]) + } + } + + // "yyyy" with "G" can't parse negative year or year 0000. + val formatter2 = DateFormatter("G yyyy-MM-dd", ZoneOffset.UTC) + assertParsingError(formatter2.parse("BC -1234-02-22")) + assertParsingError(formatter2.parse("AD 0000-02-22")) + + assert(formatter2.parse("BC 1234-02-22") === localDateToDays(LocalDate.of(-1233, 2, 22))) + assert(formatter2.parse("AD 1234-02-22") === localDateToDays(LocalDate.of(1234, 2, 22))) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index 082849c88669a..b2c39242aad53 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -17,14 +17,15 @@ package org.apache.spark.sql.util -import java.time.{Instant, LocalDateTime, LocalTime, ZoneOffset} +import java.time.{DateTimeException, Instant, LocalDateTime, LocalTime, ZoneOffset} import java.util.concurrent.TimeUnit import org.scalatest.Matchers -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkUpgradeException} import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{CET, PST, UTC} import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros import org.apache.spark.unsafe.types.UTF8String @@ -34,8 +35,8 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers val localDate = "2018-12-02T10:11:12.001234" val expectedMicros = Map( "UTC" -> 1543745472001234L, - "PST" -> 1543774272001234L, - "CET" -> 1543741872001234L, + PST.getId -> 1543774272001234L, + CET.getId -> 1543741872001234L, "Africa/Dakar" -> 1543745472001234L, "America/Los_Angeles" -> 1543774272001234L, "Antarctica/Vostok" -> 1543723872001234L, @@ -44,7 +45,8 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers DateTimeTestUtils.outstandingTimezonesIds.foreach { zoneId => val formatter = TimestampFormatter( "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", - DateTimeUtils.getZoneId(zoneId)) + DateTimeUtils.getZoneId(zoneId), + needVarLengthSecondFraction = true) val microsSinceEpoch = formatter.parse(localDate) assert(microsSinceEpoch === expectedMicros(zoneId)) } @@ -54,8 +56,8 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers val microsSinceEpoch = 1543745472001234L val expectedTimestamp = Map( "UTC" -> "2018-12-02T10:11:12.001234", - "PST" -> "2018-12-02T02:11:12.001234", - "CET" -> "2018-12-02T11:11:12.001234", + PST.getId -> "2018-12-02T02:11:12.001234", + CET.getId -> "2018-12-02T11:11:12.001234", "Africa/Dakar" -> "2018-12-02T10:11:12.001234", "America/Los_Angeles" -> "2018-12-02T02:11:12.001234", "Antarctica/Vostok" -> "2018-12-02T16:11:12.001234", @@ -83,9 +85,9 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers 2177456523456789L, 11858049903010203L).foreach { micros => DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => - val formatter = TimestampFormatter(pattern, zoneId) - val timestamp = formatter.format(micros) - val parsed = formatter.parse(timestamp) + val timestamp = TimestampFormatter(pattern, zoneId).format(micros) + val parsed = TimestampFormatter( + pattern, zoneId, needVarLengthSecondFraction = true).parse(timestamp) assert(micros === parsed) } } @@ -104,15 +106,16 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers "2039-01-01T01:02:03.456789", "2345-10-07T22:45:03.010203").foreach { timestamp => DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => - val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", zoneId) - val micros = formatter.parse(timestamp) - val formatted = formatter.format(micros) + val pattern = "yyyy-MM-dd'T'HH:mm:ss.SSSSSS" + val micros = TimestampFormatter( + pattern, zoneId, needVarLengthSecondFraction = true).parse(timestamp) + val formatted = TimestampFormatter(pattern, zoneId).format(micros) assert(timestamp === formatted) } } } - test(" case insensitive parsing of am and pm") { + test("case insensitive parsing of am and pm") { val formatter = TimestampFormatter("yyyy MMM dd hh:mm:ss a", ZoneOffset.UTC) val micros = formatter.parse("2009 Mar 20 11:30:01 am") assert(micros === TimeUnit.SECONDS.toMicros( @@ -159,7 +162,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers test("parsing timestamp strings with various seconds fractions") { DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => def check(pattern: String, input: String, reference: String): Unit = { - val formatter = TimestampFormatter(pattern, zoneId) + val formatter = TimestampFormatter(pattern, zoneId, needVarLengthSecondFraction = true) val expected = DateTimeUtils.stringToTimestamp( UTF8String.fromString(reference), zoneId).get val actual = formatter.parse(input) @@ -184,7 +187,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers "2019-10-14T09:39:07.1", "2019-10-14T09:39:07.1") try { - TimestampFormatter("yyyy/MM/dd HH_mm_ss.SSSSSS", zoneId) + TimestampFormatter("yyyy/MM/dd HH_mm_ss.SSSSSS", zoneId, true) .parse("2019/11/14 20#25#30.123456") fail("Expected to throw an exception for the invalid input") } catch { @@ -233,4 +236,30 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers "2019-10-14 09:39:07") } } + + test("SPARK-30958: parse timestamp with negative year") { + val formatter1 = TimestampFormatter("yyyy-MM-dd HH:mm:ss", ZoneOffset.UTC, true) + assert(formatter1.parse("-1234-02-22 02:22:22") === instantToMicros( + LocalDateTime.of(-1234, 2, 22, 2, 22, 22).toInstant(ZoneOffset.UTC))) + + def assertParsingError(f: => Unit): Unit = { + intercept[Exception](f) match { + case e: SparkUpgradeException => + assert(e.getCause.isInstanceOf[DateTimeException]) + case e => + assert(e.isInstanceOf[DateTimeException]) + } + } + + // "yyyy" with "G" can't parse negative year or year 0000. + val formatter2 = TimestampFormatter("G yyyy-MM-dd HH:mm:ss", UTC, true) + assertParsingError(formatter2.parse("BC -1234-02-22 02:22:22")) + assertParsingError(formatter2.parse("AC 0000-02-22 02:22:22")) + + assert(formatter2.parse("BC 1234-02-22 02:22:22") === instantToMicros( + LocalDateTime.of(-1233, 2, 22, 2, 22, 22).toInstant(ZoneOffset.UTC))) + assert(formatter2.parse("AD 1234-02-22 02:22:22") === instantToMicros( + LocalDateTime.of(1234, 2, 22, 2, 22, 22).toInstant(ZoneOffset.UTC))) + + } } diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index d8071e7bbdb35..147a77ff098d0 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -2,66 +2,66 @@ Benchmark to measure CSV read/write performance ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 44297 44515 373 0.0 885948.7 1.0X +One quoted string 24907 29374 NaN 0.0 498130.5 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 196720 197783 1560 0.0 196719.8 1.0X -Select 100 columns 46691 46861 219 0.0 46691.4 4.2X -Select one column 36811 36922 111 0.0 36811.3 5.3X -count() 8520 8610 106 0.1 8520.5 23.1X -Select 100 columns, one bad input field 67914 67994 136 0.0 67914.0 2.9X -Select 100 columns, corrupt record field 77272 77445 214 0.0 77272.0 2.5X +Select 1000 columns 62811 63690 1416 0.0 62811.4 1.0X +Select 100 columns 23839 24064 230 0.0 23839.5 2.6X +Select one column 19936 20641 827 0.1 19936.4 3.2X +count() 4174 4380 206 0.2 4174.4 15.0X +Select 100 columns, one bad input field 41015 42380 1688 0.0 41015.4 1.5X +Select 100 columns, corrupt record field 46281 46338 93 0.0 46280.5 1.4X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 25965 26054 103 0.4 2596.5 1.0X -Select 1 column + count() 18591 18666 91 0.5 1859.1 1.4X -count() 6102 6119 18 1.6 610.2 4.3X +Select 10 columns + count() 10810 10997 163 0.9 1081.0 1.0X +Select 1 column + count() 7608 7641 47 1.3 760.8 1.4X +count() 2415 2462 77 4.1 241.5 4.5X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2142 2161 17 4.7 214.2 1.0X -to_csv(timestamp) 14744 14950 182 0.7 1474.4 0.1X -write timestamps to files 12078 12202 175 0.8 1207.8 0.2X -Create a dataset of dates 2275 2291 18 4.4 227.5 0.9X -to_csv(date) 11407 11464 51 0.9 1140.7 0.2X -write dates to files 7638 7702 90 1.3 763.8 0.3X +Create a dataset of timestamps 874 914 37 11.4 87.4 1.0X +to_csv(timestamp) 7051 7223 250 1.4 705.1 0.1X +write timestamps to files 6712 6741 31 1.5 671.2 0.1X +Create a dataset of dates 909 945 35 11.0 90.9 1.0X +to_csv(date) 4222 4231 8 2.4 422.2 0.2X +write dates to files 3799 3813 14 2.6 379.9 0.2X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2578 2590 10 3.9 257.8 1.0X -read timestamps from files 60103 60694 512 0.2 6010.3 0.0X -infer timestamps from files 107871 108268 351 0.1 10787.1 0.0X -read date text from files 2306 2310 4 4.3 230.6 1.1X -read date from files 47415 47657 367 0.2 4741.5 0.1X -infer date from files 35261 35447 164 0.3 3526.1 0.1X -timestamp strings 3045 3056 11 3.3 304.5 0.8X -parse timestamps from Dataset[String] 62221 63173 849 0.2 6222.1 0.0X -infer timestamps from Dataset[String] 118838 119629 697 0.1 11883.8 0.0X -date strings 3459 3481 19 2.9 345.9 0.7X -parse dates from Dataset[String] 51026 51447 503 0.2 5102.6 0.1X -from_csv(timestamp) 60738 61818 936 0.2 6073.8 0.0X -from_csv(date) 46012 46278 370 0.2 4601.2 0.1X +read timestamp text from files 1342 1364 35 7.5 134.2 1.0X +read timestamps from files 20300 20473 247 0.5 2030.0 0.1X +infer timestamps from files 40705 40744 54 0.2 4070.5 0.0X +read date text from files 1146 1151 6 8.7 114.6 1.2X +read date from files 12278 12408 117 0.8 1227.8 0.1X +infer date from files 12734 12872 220 0.8 1273.4 0.1X +timestamp strings 1467 1482 15 6.8 146.7 0.9X +parse timestamps from Dataset[String] 21708 22234 477 0.5 2170.8 0.1X +infer timestamps from Dataset[String] 42357 43253 922 0.2 4235.7 0.0X +date strings 1512 1532 18 6.6 151.2 0.9X +parse dates from Dataset[String] 13436 13470 33 0.7 1343.6 0.1X +from_csv(timestamp) 20390 20486 95 0.5 2039.0 0.1X +from_csv(date) 12592 12693 139 0.8 1259.2 0.1X -OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 11889 11945 52 0.0 118893.1 1.0X -pushdown disabled 11790 11860 115 0.0 117902.3 1.0X -w/ filters 1240 1278 33 0.1 12400.8 9.6X +w/o filters 12535 12606 67 0.0 125348.8 1.0X +pushdown disabled 12611 12672 91 0.0 126112.9 1.0X +w/ filters 1093 1099 11 0.1 10928.3 11.5X diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index b3ba69c9eb6b1..498ca4caa0e45 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -2,66 +2,66 @@ Benchmark to measure CSV read/write performance ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 51602 51659 59 0.0 1032039.4 1.0X +One quoted string 24073 24109 33 0.0 481463.5 1.0X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 191926 192879 1615 0.0 191925.6 1.0X -Select 100 columns 46766 46846 69 0.0 46766.1 4.1X -Select one column 35877 35930 83 0.0 35876.8 5.3X -count() 11186 11262 65 0.1 11186.0 17.2X -Select 100 columns, one bad input field 59943 60107 232 0.0 59943.0 3.2X -Select 100 columns, corrupt record field 73062 73406 479 0.0 73062.2 2.6X +Select 1000 columns 58415 59611 2071 0.0 58414.8 1.0X +Select 100 columns 22568 23020 594 0.0 22568.0 2.6X +Select one column 18995 19058 99 0.1 18995.0 3.1X +count() 5301 5332 30 0.2 5300.9 11.0X +Select 100 columns, one bad input field 39736 40153 361 0.0 39736.1 1.5X +Select 100 columns, corrupt record field 47195 47826 590 0.0 47195.2 1.2X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 22389 22447 87 0.4 2238.9 1.0X -Select 1 column + count() 14844 14890 43 0.7 1484.4 1.5X -count() 5519 5538 18 1.8 551.9 4.1X +Select 10 columns + count() 9884 9904 25 1.0 988.4 1.0X +Select 1 column + count() 6794 6835 46 1.5 679.4 1.5X +count() 2060 2065 5 4.9 206.0 4.8X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 1949 1977 25 5.1 194.9 1.0X -to_csv(timestamp) 14944 15702 714 0.7 1494.4 0.1X -write timestamps to files 12983 12998 14 0.8 1298.3 0.2X -Create a dataset of dates 2156 2164 7 4.6 215.6 0.9X -to_csv(date) 9675 9709 41 1.0 967.5 0.2X -write dates to files 7880 7897 15 1.3 788.0 0.2X +Create a dataset of timestamps 717 732 18 14.0 71.7 1.0X +to_csv(timestamp) 6994 7100 121 1.4 699.4 0.1X +write timestamps to files 6417 6435 27 1.6 641.7 0.1X +Create a dataset of dates 827 855 24 12.1 82.7 0.9X +to_csv(date) 4408 4438 32 2.3 440.8 0.2X +write dates to files 3738 3758 28 2.7 373.8 0.2X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2235 2245 10 4.5 223.5 1.0X -read timestamps from files 54490 54690 283 0.2 5449.0 0.0X -infer timestamps from files 104501 104737 236 0.1 10450.1 0.0X -read date text from files 2035 2040 6 4.9 203.5 1.1X -read date from files 39650 39707 52 0.3 3965.0 0.1X -infer date from files 29235 29363 164 0.3 2923.5 0.1X -timestamp strings 3412 3426 18 2.9 341.2 0.7X -parse timestamps from Dataset[String] 66864 67804 981 0.1 6686.4 0.0X -infer timestamps from Dataset[String] 118780 119284 837 0.1 11878.0 0.0X -date strings 3730 3734 4 2.7 373.0 0.6X -parse dates from Dataset[String] 48728 49071 309 0.2 4872.8 0.0X -from_csv(timestamp) 62294 62493 260 0.2 6229.4 0.0X -from_csv(date) 44581 44665 117 0.2 4458.1 0.1X +read timestamp text from files 1121 1176 52 8.9 112.1 1.0X +read timestamps from files 21298 21366 105 0.5 2129.8 0.1X +infer timestamps from files 41008 41051 39 0.2 4100.8 0.0X +read date text from files 962 967 5 10.4 96.2 1.2X +read date from files 11749 11772 22 0.9 1174.9 0.1X +infer date from files 12426 12459 29 0.8 1242.6 0.1X +timestamp strings 1508 1519 9 6.6 150.8 0.7X +parse timestamps from Dataset[String] 21674 21997 455 0.5 2167.4 0.1X +infer timestamps from Dataset[String] 42141 42230 105 0.2 4214.1 0.0X +date strings 1694 1701 8 5.9 169.4 0.7X +parse dates from Dataset[String] 12929 12951 25 0.8 1292.9 0.1X +from_csv(timestamp) 20603 20786 166 0.5 2060.3 0.1X +from_csv(date) 12325 12338 12 0.8 1232.5 0.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -w/o filters 12557 12634 78 0.0 125572.9 1.0X -pushdown disabled 12449 12509 65 0.0 124486.4 1.0X -w/ filters 1372 1393 18 0.1 13724.8 9.1X +w/o filters 12455 12474 22 0.0 124553.8 1.0X +pushdown disabled 12462 12486 29 0.0 124624.9 1.0X +w/ filters 1073 1092 18 0.1 10727.6 11.6X diff --git a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt index 883f9de74c22f..1323a013332d2 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt @@ -2,428 +2,429 @@ Extract components ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 221 232 16 45.3 22.1 1.0X -cast to timestamp wholestage on 213 256 71 46.9 21.3 1.0X +cast to timestamp wholestage off 440 459 28 22.7 44.0 1.0X +cast to timestamp wholestage on 384 407 15 26.1 38.4 1.1X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 863 961 139 11.6 86.3 1.0X -year of timestamp wholestage on 783 821 26 12.8 78.3 1.1X +year of timestamp wholestage off 1334 1352 27 7.5 133.4 1.0X +year of timestamp wholestage on 1301 1309 9 7.7 130.1 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 1008 1013 7 9.9 100.8 1.0X -quarter of timestamp wholestage on 926 963 36 10.8 92.6 1.1X +quarter of timestamp wholestage off 1623 1636 18 6.2 162.3 1.0X +quarter of timestamp wholestage on 1592 1603 9 6.3 159.2 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 794 813 27 12.6 79.4 1.0X -month of timestamp wholestage on 737 758 23 13.6 73.7 1.1X +month of timestamp wholestage off 1246 1250 5 8.0 124.6 1.0X +month of timestamp wholestage on 1277 1290 15 7.8 127.7 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 1063 1076 18 9.4 106.3 1.0X -weekofyear of timestamp wholestage on 1070 1366 305 9.3 107.0 1.0X +weekofyear of timestamp wholestage off 1881 1896 20 5.3 188.1 1.0X +weekofyear of timestamp wholestage on 1865 1879 16 5.4 186.5 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 847 854 9 11.8 84.7 1.0X -day of timestamp wholestage on 799 836 37 12.5 79.9 1.1X +day of timestamp wholestage off 1223 1249 37 8.2 122.3 1.0X +day of timestamp wholestage on 1261 1276 19 7.9 126.1 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 816 830 20 12.2 81.6 1.0X -dayofyear of timestamp wholestage on 856 925 62 11.7 85.6 1.0X +dayofyear of timestamp wholestage off 1291 1312 29 7.7 129.1 1.0X +dayofyear of timestamp wholestage on 1284 1290 6 7.8 128.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 1150 1159 12 8.7 115.0 1.0X -dayofmonth of timestamp wholestage on 945 1165 308 10.6 94.5 1.2X +dayofmonth of timestamp wholestage off 1241 1246 7 8.1 124.1 1.0X +dayofmonth of timestamp wholestage on 1268 1276 6 7.9 126.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 1062 1074 16 9.4 106.2 1.0X -dayofweek of timestamp wholestage on 942 1029 74 10.6 94.2 1.1X +dayofweek of timestamp wholestage off 1402 1408 9 7.1 140.2 1.0X +dayofweek of timestamp wholestage on 1414 1423 8 7.1 141.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 870 872 3 11.5 87.0 1.0X -weekday of timestamp wholestage on 821 855 21 12.2 82.1 1.1X +weekday of timestamp wholestage off 1349 1350 2 7.4 134.9 1.0X +weekday of timestamp wholestage on 1344 1352 5 7.4 134.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 578 582 5 17.3 57.8 1.0X -hour of timestamp wholestage on 533 544 14 18.8 53.3 1.1X +hour of timestamp wholestage off 974 974 0 10.3 97.4 1.0X +hour of timestamp wholestage on 1009 1019 15 9.9 100.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 611 613 3 16.4 61.1 1.0X -minute of timestamp wholestage on 551 557 8 18.2 55.1 1.1X +minute of timestamp wholestage off 1024 1028 6 9.8 102.4 1.0X +minute of timestamp wholestage on 978 992 11 10.2 97.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 705 716 16 14.2 70.5 1.0X -second of timestamp wholestage on 569 669 99 17.6 56.9 1.2X +second of timestamp wholestage off 1023 1026 4 9.8 102.3 1.0X +second of timestamp wholestage on 1008 1024 14 9.9 100.8 1.0X ================================================================================================ Current date and time ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 177 177 0 56.6 17.7 1.0X -current_date wholestage on 164 180 13 60.9 16.4 1.1X +current_date wholestage off 296 299 5 33.8 29.6 1.0X +current_date wholestage on 316 318 2 31.7 31.6 0.9X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 182 182 1 55.1 18.2 1.0X -current_timestamp wholestage on 168 194 53 59.4 16.8 1.1X +current_timestamp wholestage off 310 345 49 32.2 31.0 1.0X +current_timestamp wholestage on 316 332 11 31.7 31.6 1.0X ================================================================================================ Date arithmetic ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 615 632 24 16.3 61.5 1.0X -cast to date wholestage on 596 660 40 16.8 59.6 1.0X +cast to date wholestage off 1067 1071 7 9.4 106.7 1.0X +cast to date wholestage on 1050 1072 20 9.5 105.0 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 717 728 16 13.9 71.7 1.0X -last_day wholestage on 764 834 42 13.1 76.4 0.9X +last_day wholestage off 1240 1245 8 8.1 124.0 1.0X +last_day wholestage on 1273 1290 14 7.9 127.3 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 677 677 0 14.8 67.7 1.0X -next_day wholestage on 645 796 140 15.5 64.5 1.1X +next_day wholestage off 1121 1130 13 8.9 112.1 1.0X +next_day wholestage on 1099 1106 11 9.1 109.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 637 663 37 15.7 63.7 1.0X -date_add wholestage on 659 675 16 15.2 65.9 1.0X +date_add wholestage off 1045 1051 8 9.6 104.5 1.0X +date_add wholestage on 1078 1086 16 9.3 107.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 630 682 73 15.9 63.0 1.0X -date_sub wholestage on 720 762 37 13.9 72.0 0.9X +date_sub wholestage off 1049 1050 1 9.5 104.9 1.0X +date_sub wholestage on 1078 1083 4 9.3 107.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 1064 1088 35 9.4 106.4 1.0X -add_months wholestage on 883 909 25 11.3 88.3 1.2X +add_months wholestage off 1411 1415 7 7.1 141.1 1.0X +add_months wholestage on 1435 1447 13 7.0 143.5 1.0X ================================================================================================ Formatting dates ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 3667 3854 265 2.7 366.7 1.0X -format date wholestage on 3355 3548 195 3.0 335.5 1.1X +format date wholestage off 5736 5786 71 1.7 573.6 1.0X +format date wholestage on 5550 5575 29 1.8 555.0 1.0X ================================================================================================ Formatting timestamps ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 4664 4704 57 2.1 466.4 1.0X -from_unixtime wholestage on 4596 4772 193 2.2 459.6 1.0X +from_unixtime wholestage off 7378 7379 1 1.4 737.8 1.0X +from_unixtime wholestage on 7441 7465 24 1.3 744.1 1.0X ================================================================================================ Convert timestamps ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 810 819 13 12.3 81.0 1.0X -from_utc_timestamp wholestage on 795 814 16 12.6 79.5 1.0X +from_utc_timestamp wholestage off 1238 1260 30 8.1 123.8 1.0X +from_utc_timestamp wholestage on 1300 1308 5 7.7 130.0 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 1130 1146 22 8.8 113.0 1.0X -to_utc_timestamp wholestage on 1034 1063 40 9.7 103.4 1.1X +to_utc_timestamp wholestage off 1790 1793 5 5.6 179.0 1.0X +to_utc_timestamp wholestage on 1737 1748 13 5.8 173.7 1.0X ================================================================================================ Intervals ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 192 194 2 52.1 19.2 1.0X -cast interval wholestage on 172 180 5 58.2 17.2 1.1X +cast interval wholestage off 347 350 4 28.8 34.7 1.0X +cast interval wholestage on 327 329 4 30.6 32.7 1.1X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1070 1082 17 9.3 107.0 1.0X -datediff wholestage on 1039 1059 13 9.6 103.9 1.0X +datediff wholestage off 1836 1860 33 5.4 183.6 1.0X +datediff wholestage on 1811 1826 16 5.5 181.1 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 3118 3137 27 3.2 311.8 1.0X -months_between wholestage on 3079 3122 48 3.2 307.9 1.0X +months_between wholestage off 5613 5617 6 1.8 561.3 1.0X +months_between wholestage on 5590 5624 35 1.8 559.0 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 1147 1157 14 0.9 1146.6 1.0X -window wholestage on 16698 17304 715 0.1 16698.2 0.1X +window wholestage off 2222 2308 122 0.5 2222.2 1.0X +window wholestage on 48721 48828 101 0.0 48720.7 0.0X ================================================================================================ Truncation ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 1478 1485 9 6.8 147.8 1.0X -date_trunc YEAR wholestage on 1458 1587 213 6.9 145.8 1.0X +date_trunc YEAR wholestage off 2482 2503 30 4.0 248.2 1.0X +date_trunc YEAR wholestage on 2520 2532 12 4.0 252.0 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 1467 1481 20 6.8 146.7 1.0X -date_trunc YYYY wholestage on 1457 1513 52 6.9 145.7 1.0X +date_trunc YYYY wholestage off 2487 2493 9 4.0 248.7 1.0X +date_trunc YYYY wholestage on 2521 2527 8 4.0 252.1 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 1551 1556 7 6.4 155.1 1.0X -date_trunc YY wholestage on 1468 1538 78 6.8 146.8 1.1X +date_trunc YY wholestage off 2479 2492 18 4.0 247.9 1.0X +date_trunc YY wholestage on 2519 2529 8 4.0 251.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 1481 1493 16 6.8 148.1 1.0X -date_trunc MON wholestage on 1476 1501 28 6.8 147.6 1.0X +date_trunc MON wholestage off 2504 2519 21 4.0 250.4 1.0X +date_trunc MON wholestage on 2549 2563 15 3.9 254.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 1470 1477 10 6.8 147.0 1.0X -date_trunc MONTH wholestage on 1449 1610 260 6.9 144.9 1.0X +date_trunc MONTH wholestage off 2517 2527 14 4.0 251.7 1.0X +date_trunc MONTH wholestage on 2538 2547 11 3.9 253.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 1487 1495 12 6.7 148.7 1.0X -date_trunc MM wholestage on 1467 1489 17 6.8 146.7 1.0X +date_trunc MM wholestage off 2506 2507 2 4.0 250.6 1.0X +date_trunc MM wholestage on 2536 2546 10 3.9 253.6 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 1321 1328 10 7.6 132.1 1.0X -date_trunc DAY wholestage on 1279 1306 28 7.8 127.9 1.0X +date_trunc DAY wholestage off 2443 2454 17 4.1 244.3 1.0X +date_trunc DAY wholestage on 2468 2479 12 4.1 246.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 1366 1368 3 7.3 136.6 1.0X -date_trunc DD wholestage on 1297 1314 18 7.7 129.7 1.1X +date_trunc DD wholestage off 2440 2440 0 4.1 244.0 1.0X +date_trunc DD wholestage on 2459 2481 23 4.1 245.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 1338 1354 23 7.5 133.8 1.0X -date_trunc HOUR wholestage on 1403 1482 71 7.1 140.3 1.0X +date_trunc HOUR wholestage off 2452 2453 2 4.1 245.2 1.0X +date_trunc HOUR wholestage on 2428 2435 5 4.1 242.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 241 242 2 41.5 24.1 1.0X -date_trunc MINUTE wholestage on 196 204 6 51.0 19.6 1.2X +date_trunc MINUTE wholestage off 359 359 1 27.8 35.9 1.0X +date_trunc MINUTE wholestage on 382 388 8 26.2 38.2 0.9X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 211 222 15 47.3 21.1 1.0X -date_trunc SECOND wholestage on 205 243 64 48.8 20.5 1.0X +date_trunc SECOND wholestage off 382 386 6 26.2 38.2 1.0X +date_trunc SECOND wholestage on 379 382 5 26.4 37.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 1492 1515 32 6.7 149.2 1.0X -date_trunc WEEK wholestage on 1465 1490 39 6.8 146.5 1.0X +date_trunc WEEK wholestage off 2427 2429 3 4.1 242.7 1.0X +date_trunc WEEK wholestage on 2427 2441 12 4.1 242.7 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 1908 1982 104 5.2 190.8 1.0X -date_trunc QUARTER wholestage on 1879 2020 99 5.3 187.9 1.0X +date_trunc QUARTER wholestage off 3551 3583 46 2.8 355.1 1.0X +date_trunc QUARTER wholestage on 3517 3526 6 2.8 351.7 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 326 419 132 30.7 32.6 1.0X -trunc year wholestage on 187 199 8 53.4 18.7 1.7X +trunc year wholestage off 312 313 1 32.0 31.2 1.0X +trunc year wholestage on 319 326 7 31.3 31.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 194 194 0 51.6 19.4 1.0X -trunc yyyy wholestage on 192 207 10 52.1 19.2 1.0X +trunc yyyy wholestage off 318 318 0 31.5 31.8 1.0X +trunc yyyy wholestage on 317 328 17 31.5 31.7 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 177 180 4 56.5 17.7 1.0X -trunc yy wholestage on 185 193 9 53.9 18.5 1.0X +trunc yy wholestage off 315 322 10 31.8 31.5 1.0X +trunc yy wholestage on 318 328 9 31.5 31.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 208 210 3 48.0 20.8 1.0X -trunc mon wholestage on 195 204 8 51.3 19.5 1.1X +trunc mon wholestage off 311 313 2 32.1 31.1 1.0X +trunc mon wholestage on 322 326 6 31.0 32.2 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 182 185 4 55.1 18.2 1.0X -trunc month wholestage on 185 222 70 54.0 18.5 1.0X +trunc month wholestage off 310 318 11 32.2 31.0 1.0X +trunc month wholestage on 319 326 8 31.3 31.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 208 218 14 48.0 20.8 1.0X -trunc mm wholestage on 197 206 10 50.8 19.7 1.1X +trunc mm wholestage off 311 314 4 32.2 31.1 1.0X +trunc mm wholestage on 318 324 7 31.4 31.8 1.0X ================================================================================================ Parsing ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 121 130 13 8.3 120.7 1.0X -to timestamp str wholestage on 128 134 6 7.8 128.4 0.9X +to timestamp str wholestage off 166 167 2 6.0 165.7 1.0X +to timestamp str wholestage on 158 162 4 6.3 158.2 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 1201 1272 100 0.8 1200.8 1.0X -to_timestamp wholestage on 973 1019 53 1.0 972.7 1.2X +to_timestamp wholestage off 1487 1488 1 0.7 1487.1 1.0X +to_timestamp wholestage on 1418 1422 4 0.7 1418.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 1002 1077 106 1.0 1001.8 1.0X -to_unix_timestamp wholestage on 988 1079 100 1.0 987.8 1.0X +to_unix_timestamp wholestage off 1409 1420 15 0.7 1409.2 1.0X +to_unix_timestamp wholestage on 1422 1428 8 0.7 1422.1 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 157 165 12 6.4 157.2 1.0X -to date str wholestage on 137 140 3 7.3 137.0 1.1X +to date str wholestage off 207 207 0 4.8 206.9 1.0X +to date str wholestage on 209 210 1 4.8 209.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 1597 1614 25 0.6 1597.0 1.0X -to_date wholestage on 1503 1573 58 0.7 1502.8 1.1X +to_date wholestage off 3378 3398 28 0.3 3377.7 1.0X +to_date wholestage on 3372 3379 8 0.3 3372.2 1.0X ================================================================================================ Conversion from/to external types ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz -To/from java.sql.Timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Timestamp 197 202 6 25.4 39.4 1.0X -Collect longs 794 911 110 6.3 158.7 0.2X -Collect timestamps 1030 1079 51 4.9 205.9 0.2X +From java.sql.Date 427 443 23 11.7 85.4 1.0X +From java.sql.Timestamp 361 369 11 13.8 72.3 1.2X +Collect longs 1942 2044 170 2.6 388.3 0.2X +Collect timestamps 1891 2093 192 2.6 378.2 0.2X diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt index 2ea26c327afae..77b6f833221e5 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt @@ -2,428 +2,429 @@ Extract components ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 230 240 13 43.5 23.0 1.0X -cast to timestamp wholestage on 194 208 20 51.4 19.4 1.2X +cast to timestamp wholestage off 410 428 25 24.4 41.0 1.0X +cast to timestamp wholestage on 364 379 17 27.5 36.4 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 806 822 22 12.4 80.6 1.0X -year of timestamp wholestage on 748 761 13 13.4 74.8 1.1X +year of timestamp wholestage off 1308 1318 15 7.6 130.8 1.0X +year of timestamp wholestage on 1204 1234 19 8.3 120.4 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 828 832 5 12.1 82.8 1.0X -quarter of timestamp wholestage on 821 858 45 12.2 82.1 1.0X +quarter of timestamp wholestage off 1439 1442 6 7.0 143.9 1.0X +quarter of timestamp wholestage on 1374 1385 9 7.3 137.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 709 713 5 14.1 70.9 1.0X -month of timestamp wholestage on 714 722 10 14.0 71.4 1.0X +month of timestamp wholestage off 1228 1234 8 8.1 122.8 1.0X +month of timestamp wholestage on 1225 1241 25 8.2 122.5 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 1217 1220 4 8.2 121.7 1.0X -weekofyear of timestamp wholestage on 1019 1043 24 9.8 101.9 1.2X +weekofyear of timestamp wholestage off 1878 1880 3 5.3 187.8 1.0X +weekofyear of timestamp wholestage on 1864 1870 6 5.4 186.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 706 712 9 14.2 70.6 1.0X -day of timestamp wholestage on 694 704 8 14.4 69.4 1.0X +day of timestamp wholestage off 1218 1220 3 8.2 121.8 1.0X +day of timestamp wholestage on 1217 1226 10 8.2 121.7 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 717 728 16 14.0 71.7 1.0X -dayofyear of timestamp wholestage on 724 736 10 13.8 72.4 1.0X +dayofyear of timestamp wholestage off 1274 1282 12 7.8 127.4 1.0X +dayofyear of timestamp wholestage on 1263 1274 14 7.9 126.3 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 719 721 3 13.9 71.9 1.0X -dayofmonth of timestamp wholestage on 706 714 8 14.2 70.6 1.0X +dayofmonth of timestamp wholestage off 1221 1226 6 8.2 122.1 1.0X +dayofmonth of timestamp wholestage on 1215 1222 9 8.2 121.5 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 904 979 106 11.1 90.4 1.0X -dayofweek of timestamp wholestage on 796 819 14 12.6 79.6 1.1X +dayofweek of timestamp wholestage off 1404 1431 38 7.1 140.4 1.0X +dayofweek of timestamp wholestage on 1374 1381 8 7.3 137.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 813 819 9 12.3 81.3 1.0X -weekday of timestamp wholestage on 788 809 13 12.7 78.8 1.0X +weekday of timestamp wholestage off 1342 1350 12 7.5 134.2 1.0X +weekday of timestamp wholestage on 1311 1317 5 7.6 131.1 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 567 582 21 17.6 56.7 1.0X -hour of timestamp wholestage on 539 551 11 18.6 53.9 1.1X +hour of timestamp wholestage off 997 1000 4 10.0 99.7 1.0X +hour of timestamp wholestage on 959 969 8 10.4 95.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 554 563 13 18.1 55.4 1.0X -minute of timestamp wholestage on 520 531 7 19.2 52.0 1.1X +minute of timestamp wholestage off 968 970 2 10.3 96.8 1.0X +minute of timestamp wholestage on 952 960 8 10.5 95.2 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 651 654 5 15.4 65.1 1.0X -second of timestamp wholestage on 535 546 12 18.7 53.5 1.2X +second of timestamp wholestage off 971 974 5 10.3 97.1 1.0X +second of timestamp wholestage on 952 953 0 10.5 95.2 1.0X ================================================================================================ Current date and time ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 172 175 4 58.0 17.2 1.0X -current_date wholestage on 174 177 3 57.5 17.4 1.0X +current_date wholestage off 277 282 7 36.1 27.7 1.0X +current_date wholestage on 283 290 11 35.3 28.3 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 184 185 0 54.3 18.4 1.0X -current_timestamp wholestage on 196 212 14 51.0 19.6 0.9X +current_timestamp wholestage off 302 324 31 33.1 30.2 1.0X +current_timestamp wholestage on 281 292 13 35.6 28.1 1.1X ================================================================================================ Date arithmetic ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 652 662 13 15.3 65.2 1.0X -cast to date wholestage on 645 690 29 15.5 64.5 1.0X +cast to date wholestage off 1064 1066 2 9.4 106.4 1.0X +cast to date wholestage on 1002 1010 8 10.0 100.2 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 801 808 11 12.5 80.1 1.0X -last_day wholestage on 765 829 37 13.1 76.5 1.0X +last_day wholestage off 1247 1250 4 8.0 124.7 1.0X +last_day wholestage on 1238 1245 4 8.1 123.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 676 676 0 14.8 67.6 1.0X -next_day wholestage on 674 708 33 14.8 67.4 1.0X +next_day wholestage off 1112 1113 1 9.0 111.2 1.0X +next_day wholestage on 1055 1059 5 9.5 105.5 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 644 646 2 15.5 64.4 1.0X -date_add wholestage on 640 661 32 15.6 64.0 1.0X +date_add wholestage off 1049 1051 3 9.5 104.9 1.0X +date_add wholestage on 1036 1043 9 9.6 103.6 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 704 718 20 14.2 70.4 1.0X -date_sub wholestage on 684 728 34 14.6 68.4 1.0X +date_sub wholestage off 1043 1063 29 9.6 104.3 1.0X +date_sub wholestage on 1036 1041 5 9.6 103.6 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 1011 1017 8 9.9 101.1 1.0X -add_months wholestage on 837 860 25 11.9 83.7 1.2X +add_months wholestage off 1371 1373 2 7.3 137.1 1.0X +add_months wholestage on 1402 1409 10 7.1 140.2 1.0X ================================================================================================ Formatting dates ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 3467 3591 176 2.9 346.7 1.0X -format date wholestage on 3417 3482 66 2.9 341.7 1.0X +format date wholestage off 5741 5862 171 1.7 574.1 1.0X +format date wholestage on 5589 5600 19 1.8 558.9 1.0X ================================================================================================ Formatting timestamps ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 4823 4850 38 2.1 482.3 1.0X -from_unixtime wholestage on 4774 4811 29 2.1 477.4 1.0X +from_unixtime wholestage off 8681 8684 4 1.2 868.1 1.0X +from_unixtime wholestage on 8669 8707 33 1.2 866.9 1.0X ================================================================================================ Convert timestamps ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 703 717 20 14.2 70.3 1.0X -from_utc_timestamp wholestage on 665 671 7 15.0 66.5 1.1X +from_utc_timestamp wholestage off 1096 1101 7 9.1 109.6 1.0X +from_utc_timestamp wholestage on 1108 1116 6 9.0 110.8 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 1000 1016 22 10.0 100.0 1.0X -to_utc_timestamp wholestage on 917 933 11 10.9 91.7 1.1X +to_utc_timestamp wholestage off 1626 1638 16 6.1 162.6 1.0X +to_utc_timestamp wholestage on 1594 1606 10 6.3 159.4 1.0X ================================================================================================ Intervals ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 195 198 3 51.2 19.5 1.0X -cast interval wholestage on 176 181 5 56.9 17.6 1.1X +cast interval wholestage off 331 335 6 30.2 33.1 1.0X +cast interval wholestage on 298 305 5 33.5 29.8 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1022 1025 5 9.8 102.2 1.0X -datediff wholestage on 1003 1015 10 10.0 100.3 1.0X +datediff wholestage off 1821 1822 2 5.5 182.1 1.0X +datediff wholestage on 1759 1763 3 5.7 175.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 2562 2579 24 3.9 256.2 1.0X -months_between wholestage on 2503 2524 17 4.0 250.3 1.0X +months_between wholestage off 4752 4770 25 2.1 475.2 1.0X +months_between wholestage on 4740 4746 5 2.1 474.0 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 1067 1077 15 0.9 1066.6 1.0X -window wholestage on 14503 19089 NaN 0.1 14502.6 0.1X +window wholestage off 1936 2049 160 0.5 1935.7 1.0X +window wholestage on 44821 44850 44 0.0 44820.6 0.0X ================================================================================================ Truncation ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 1497 1543 64 6.7 149.7 1.0X -date_trunc YEAR wholestage on 1412 1533 138 7.1 141.2 1.1X +date_trunc YEAR wholestage off 2382 2384 3 4.2 238.2 1.0X +date_trunc YEAR wholestage on 2309 2316 7 4.3 230.9 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 1725 1729 5 5.8 172.5 1.0X -date_trunc YYYY wholestage on 1413 1481 56 7.1 141.3 1.2X +date_trunc YYYY wholestage off 2368 2375 10 4.2 236.8 1.0X +date_trunc YYYY wholestage on 2304 2313 7 4.3 230.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 1456 1459 5 6.9 145.6 1.0X -date_trunc YY wholestage on 1369 1380 7 7.3 136.9 1.1X +date_trunc YY wholestage off 2381 2396 22 4.2 238.1 1.0X +date_trunc YY wholestage on 2303 2305 1 4.3 230.3 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 1459 1471 17 6.9 145.9 1.0X -date_trunc MON wholestage on 1359 1377 21 7.4 135.9 1.1X +date_trunc MON wholestage off 2325 2333 11 4.3 232.5 1.0X +date_trunc MON wholestage on 2293 2301 6 4.4 229.3 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 1455 1480 35 6.9 145.5 1.0X -date_trunc MONTH wholestage on 1344 1468 170 7.4 134.4 1.1X +date_trunc MONTH wholestage off 2335 2335 0 4.3 233.5 1.0X +date_trunc MONTH wholestage on 2296 2298 2 4.4 229.6 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 1437 1463 37 7.0 143.7 1.0X -date_trunc MM wholestage on 1273 1285 9 7.9 127.3 1.1X +date_trunc MM wholestage off 2328 2329 1 4.3 232.8 1.0X +date_trunc MM wholestage on 2290 2295 4 4.4 229.0 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 1198 1209 16 8.3 119.8 1.0X -date_trunc DAY wholestage on 1137 1192 70 8.8 113.7 1.1X +date_trunc DAY wholestage off 2139 2140 1 4.7 213.9 1.0X +date_trunc DAY wholestage on 2095 2101 7 4.8 209.5 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 1201 1215 20 8.3 120.1 1.0X -date_trunc DD wholestage on 1132 1144 14 8.8 113.2 1.1X +date_trunc DD wholestage off 2143 2145 2 4.7 214.3 1.0X +date_trunc DD wholestage on 2087 2092 3 4.8 208.7 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 1236 1238 3 8.1 123.6 1.0X -date_trunc HOUR wholestage on 1134 1150 15 8.8 113.4 1.1X +date_trunc HOUR wholestage off 2201 2207 8 4.5 220.1 1.0X +date_trunc HOUR wholestage on 2176 2185 6 4.6 217.6 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 206 207 2 48.6 20.6 1.0X -date_trunc MINUTE wholestage on 185 186 2 54.0 18.5 1.1X +date_trunc MINUTE wholestage off 356 356 0 28.1 35.6 1.0X +date_trunc MINUTE wholestage on 333 344 7 30.1 33.3 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 209 210 2 47.9 20.9 1.0X -date_trunc SECOND wholestage on 174 185 7 57.4 17.4 1.2X +date_trunc SECOND wholestage off 341 342 2 29.3 34.1 1.0X +date_trunc SECOND wholestage on 335 349 18 29.9 33.5 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 1231 1293 86 8.1 123.1 1.0X -date_trunc WEEK wholestage on 1186 1219 35 8.4 118.6 1.0X +date_trunc WEEK wholestage off 2257 2257 1 4.4 225.7 1.0X +date_trunc WEEK wholestage on 2202 2211 11 4.5 220.2 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 1785 1792 10 5.6 178.5 1.0X -date_trunc QUARTER wholestage on 1671 1708 28 6.0 167.1 1.1X +date_trunc QUARTER wholestage off 3180 3181 2 3.1 318.0 1.0X +date_trunc QUARTER wholestage on 3036 3043 12 3.3 303.6 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 186 199 18 53.8 18.6 1.0X -trunc year wholestage on 161 165 5 62.3 16.1 1.2X +trunc year wholestage off 327 328 1 30.6 32.7 1.0X +trunc year wholestage on 310 325 13 32.3 31.0 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 184 184 0 54.4 18.4 1.0X -trunc yyyy wholestage on 156 162 6 64.3 15.6 1.2X +trunc yyyy wholestage off 323 327 6 31.0 32.3 1.0X +trunc yyyy wholestage on 304 317 14 32.9 30.4 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 171 174 5 58.6 17.1 1.0X -trunc yy wholestage on 161 172 10 62.1 16.1 1.1X +trunc yy wholestage off 324 325 2 30.8 32.4 1.0X +trunc yy wholestage on 308 311 3 32.4 30.8 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 175 176 1 57.1 17.5 1.0X -trunc mon wholestage on 166 167 1 60.3 16.6 1.1X +trunc mon wholestage off 323 323 1 31.0 32.3 1.0X +trunc mon wholestage on 303 310 7 33.0 30.3 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 177 186 12 56.4 17.7 1.0X -trunc month wholestage on 160 168 8 62.4 16.0 1.1X +trunc month wholestage off 322 323 0 31.0 32.2 1.0X +trunc month wholestage on 307 337 65 32.6 30.7 1.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 173 175 2 57.8 17.3 1.0X -trunc mm wholestage on 167 178 10 59.9 16.7 1.0X +trunc mm wholestage off 324 325 2 30.9 32.4 1.0X +trunc mm wholestage on 294 306 7 34.0 29.4 1.1X ================================================================================================ Parsing ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 166 168 3 6.0 166.3 1.0X -to timestamp str wholestage on 170 192 37 5.9 169.8 1.0X +to timestamp str wholestage off 216 216 1 4.6 215.7 1.0X +to timestamp str wholestage on 207 209 3 4.8 207.0 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 975 1042 95 1.0 975.1 1.0X -to_timestamp wholestage on 976 982 5 1.0 976.0 1.0X +to_timestamp wholestage off 1751 1751 1 0.6 1750.6 1.0X +to_timestamp wholestage on 1798 1832 31 0.6 1798.4 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 978 984 8 1.0 978.4 1.0X -to_unix_timestamp wholestage on 956 979 20 1.0 955.8 1.0X +to_unix_timestamp wholestage off 1770 1773 4 0.6 1770.2 1.0X +to_unix_timestamp wholestage on 1743 1754 10 0.6 1743.0 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 169 173 5 5.9 168.9 1.0X -to date str wholestage on 155 159 4 6.4 155.4 1.1X +to date str wholestage off 260 266 9 3.8 260.1 1.0X +to date str wholestage on 257 260 3 3.9 257.3 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 1583 1594 15 0.6 1583.2 1.0X -to_date wholestage on 1586 1635 52 0.6 1586.1 1.0X +to_date wholestage off 3589 3620 44 0.3 3589.0 1.0X +to_date wholestage on 3565 3579 15 0.3 3565.0 1.0X ================================================================================================ Conversion from/to external types ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz -To/from java.sql.Timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Timestamp 174 176 3 28.8 34.7 1.0X -Collect longs 945 1275 538 5.3 189.0 0.2X -Collect timestamps 1019 1422 633 4.9 203.8 0.2X +From java.sql.Date 416 420 6 12.0 83.2 1.0X +From java.sql.Timestamp 375 381 5 13.3 75.1 1.1X +Collect longs 1225 1962 1101 4.1 245.1 0.3X +Collect timestamps 2013 2829 1401 2.5 402.5 0.2X diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..2a9322a4b462a --- /dev/null +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt @@ -0,0 +1,94 @@ +================================================================================================ +Rebasing dates/timestamps in Parquet datasource +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Save dates to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, noop 21171 21171 0 4.7 211.7 1.0X +before 1582, noop 11036 11036 0 9.1 110.4 1.9X +after 1582, rebase off 34321 34321 0 2.9 343.2 0.6X +after 1582, rebase on 33269 33269 0 3.0 332.7 0.6X +before 1582, rebase off 22016 22016 0 4.5 220.2 1.0X +before 1582, rebase on 23338 23338 0 4.3 233.4 0.9X + +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Load dates from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, vec off, rebase off 12791 13089 287 7.8 127.9 1.0X +after 1582, vec off, rebase on 13203 13271 81 7.6 132.0 1.0X +after 1582, vec on, rebase off 3709 3764 49 27.0 37.1 3.4X +after 1582, vec on, rebase on 5082 5114 29 19.7 50.8 2.5X +before 1582, vec off, rebase off 13059 13153 87 7.7 130.6 1.0X +before 1582, vec off, rebase on 14211 14236 27 7.0 142.1 0.9X +before 1582, vec on, rebase off 3687 3749 72 27.1 36.9 3.5X +before 1582, vec on, rebase on 5449 5497 56 18.4 54.5 2.3X + +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Save timestamps to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, noop 2831 2831 0 35.3 28.3 1.0X +before 1582, noop 2816 2816 0 35.5 28.2 1.0X +after 1582, rebase off 15543 15543 0 6.4 155.4 0.2X +after 1582, rebase on 18391 18391 0 5.4 183.9 0.2X +before 1582, rebase off 15747 15747 0 6.4 157.5 0.2X +before 1582, rebase on 18846 18846 0 5.3 188.5 0.2X + +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Load timestamps from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, vec off, rebase off 16126 16216 78 6.2 161.3 1.0X +after 1582, vec off, rebase on 18277 18453 165 5.5 182.8 0.9X +after 1582, vec on, rebase off 5030 5067 42 19.9 50.3 3.2X +after 1582, vec on, rebase on 8553 8583 43 11.7 85.5 1.9X +before 1582, vec off, rebase off 15828 15872 39 6.3 158.3 1.0X +before 1582, vec off, rebase on 18899 18959 103 5.3 189.0 0.9X +before 1582, vec on, rebase off 4961 5009 43 20.2 49.6 3.3X +before 1582, vec on, rebase on 9099 9140 40 11.0 91.0 1.8X + + +================================================================================================ +Rebasing dates/timestamps in ORC datasource +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Save dates to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, noop 21026 21026 0 4.8 210.3 1.0X +before 1582, noop 11040 11040 0 9.1 110.4 1.9X +after 1582 28171 28171 0 3.5 281.7 0.7X +before 1582 18955 18955 0 5.3 189.5 1.1X + +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Load dates from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, vec off 10876 10931 49 9.2 108.8 1.0X +after 1582, vec on 3900 3913 20 25.6 39.0 2.8X +before 1582, vec off 11165 11174 12 9.0 111.6 1.0X +before 1582, vec on 4208 4214 7 23.8 42.1 2.6X + +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Save timestamps to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, noop 2924 2924 0 34.2 29.2 1.0X +before 1582, noop 2820 2820 0 35.5 28.2 1.0X +after 1582 22228 22228 0 4.5 222.3 0.1X +before 1582 22590 22590 0 4.4 225.9 0.1X + +OpenJDK 64-Bit Server VM 11.0.6+10-post-Ubuntu-1ubuntu118.04.1 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Load timestamps from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, vec off 13591 13658 59 7.4 135.9 1.0X +after 1582, vec on 7399 7488 126 13.5 74.0 1.8X +before 1582, vec off 14065 14096 30 7.1 140.7 1.0X +before 1582, vec on 7950 8127 249 12.6 79.5 1.7X + + diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt new file mode 100644 index 0000000000000..050950571511d --- /dev/null +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt @@ -0,0 +1,94 @@ +================================================================================================ +Rebasing dates/timestamps in Parquet datasource +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Save dates to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, noop 24114 24114 0 4.1 241.1 1.0X +before 1582, noop 10250 10250 0 9.8 102.5 2.4X +after 1582, rebase off 36672 36672 0 2.7 366.7 0.7X +after 1582, rebase on 37123 37123 0 2.7 371.2 0.6X +before 1582, rebase off 21925 21925 0 4.6 219.2 1.1X +before 1582, rebase on 22341 22341 0 4.5 223.4 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Load dates from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, vec off, rebase off 12456 12601 126 8.0 124.6 1.0X +after 1582, vec off, rebase on 13299 13336 32 7.5 133.0 0.9X +after 1582, vec on, rebase off 3623 3660 40 27.6 36.2 3.4X +after 1582, vec on, rebase on 5160 5177 15 19.4 51.6 2.4X +before 1582, vec off, rebase off 13177 13264 76 7.6 131.8 0.9X +before 1582, vec off, rebase on 14102 14149 46 7.1 141.0 0.9X +before 1582, vec on, rebase off 3649 3670 34 27.4 36.5 3.4X +before 1582, vec on, rebase on 5652 5667 15 17.7 56.5 2.2X + +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Save timestamps to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, noop 2871 2871 0 34.8 28.7 1.0X +before 1582, noop 2753 2753 0 36.3 27.5 1.0X +after 1582, rebase off 15927 15927 0 6.3 159.3 0.2X +after 1582, rebase on 19138 19138 0 5.2 191.4 0.1X +before 1582, rebase off 16137 16137 0 6.2 161.4 0.2X +before 1582, rebase on 19584 19584 0 5.1 195.8 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Load timestamps from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, vec off, rebase off 14995 15047 47 6.7 150.0 1.0X +after 1582, vec off, rebase on 18111 18146 37 5.5 181.1 0.8X +after 1582, vec on, rebase off 4837 4873 44 20.7 48.4 3.1X +after 1582, vec on, rebase on 9542 9669 111 10.5 95.4 1.6X +before 1582, vec off, rebase off 14993 15090 94 6.7 149.9 1.0X +before 1582, vec off, rebase on 18675 18712 64 5.4 186.7 0.8X +before 1582, vec on, rebase off 4908 4923 15 20.4 49.1 3.1X +before 1582, vec on, rebase on 10128 10148 19 9.9 101.3 1.5X + + +================================================================================================ +Rebasing dates/timestamps in ORC datasource +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Save dates to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, noop 23977 23977 0 4.2 239.8 1.0X +before 1582, noop 10094 10094 0 9.9 100.9 2.4X +after 1582 33115 33115 0 3.0 331.2 0.7X +before 1582 19430 19430 0 5.1 194.3 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Load dates from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, vec off 10217 10241 21 9.8 102.2 1.0X +after 1582, vec on 3671 3691 31 27.2 36.7 2.8X +before 1582, vec off 10800 10874 114 9.3 108.0 0.9X +before 1582, vec on 4118 4165 74 24.3 41.2 2.5X + +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Save timestamps to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, noop 2691 2691 0 37.2 26.9 1.0X +before 1582, noop 2743 2743 0 36.5 27.4 1.0X +after 1582 21409 21409 0 4.7 214.1 0.1X +before 1582 22554 22554 0 4.4 225.5 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_242-8u242-b08-0ubuntu3~18.04-b08 on Linux 4.15.0-1063-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Load timestamps from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +after 1582, vec off 14752 14855 103 6.8 147.5 1.0X +after 1582, vec on 8146 8185 34 12.3 81.5 1.8X +before 1582, vec off 15247 15294 46 6.6 152.5 1.0X +before 1582, vec on 8414 8466 52 11.9 84.1 1.8X + + diff --git a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt index 0873215982168..8c56c0bcc9cb0 100644 --- a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt @@ -2,118 +2,137 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 343 348 5 29.1 34.3 1.0X -MILLENNIUM of timestamp 888 899 14 11.3 88.8 0.4X -CENTURY of timestamp 835 846 15 12.0 83.5 0.4X -DECADE of timestamp 824 834 9 12.1 82.4 0.4X -YEAR of timestamp 818 824 10 12.2 81.8 0.4X -ISOYEAR of timestamp 879 890 17 11.4 87.9 0.4X -QUARTER of timestamp 965 1073 96 10.4 96.5 0.4X -MONTH of timestamp 787 793 8 12.7 78.7 0.4X -WEEK of timestamp 1277 1341 57 7.8 127.7 0.3X -DAY of timestamp 780 785 8 12.8 78.0 0.4X -DAYOFWEEK of timestamp 956 962 7 10.5 95.6 0.4X -DOW of timestamp 995 1034 36 10.1 99.5 0.3X -ISODOW of timestamp 885 898 11 11.3 88.5 0.4X -DOY of timestamp 808 810 3 12.4 80.8 0.4X -HOUR of timestamp 622 627 7 16.1 62.2 0.6X -MINUTE of timestamp 629 636 8 15.9 62.9 0.5X -SECOND of timestamp 800 819 30 12.5 80.0 0.4X -MILLISECONDS of timestamp 743 755 21 13.5 74.3 0.5X -MICROSECONDS of timestamp 692 730 53 14.5 69.2 0.5X -EPOCH of timestamp 773 798 40 12.9 77.3 0.4X +cast to timestamp 311 331 18 32.2 31.1 1.0X +MILLENNIUM of timestamp 873 893 17 11.4 87.3 0.4X +CENTURY of timestamp 869 873 5 11.5 86.9 0.4X +DECADE of timestamp 851 872 23 11.7 85.1 0.4X +YEAR of timestamp 841 856 14 11.9 84.1 0.4X +ISOYEAR of timestamp 927 938 12 10.8 92.7 0.3X +QUARTER of timestamp 959 963 6 10.4 95.9 0.3X +MONTH of timestamp 852 864 18 11.7 85.2 0.4X +WEEK of timestamp 1124 1252 112 8.9 112.4 0.3X +DAY of timestamp 848 867 19 11.8 84.8 0.4X +DAYOFWEEK of timestamp 977 987 16 10.2 97.7 0.3X +DOW of timestamp 945 964 18 10.6 94.5 0.3X +ISODOW of timestamp 924 929 5 10.8 92.4 0.3X +DOY of timestamp 852 906 67 11.7 85.2 0.4X +HOUR of timestamp 665 671 5 15.0 66.5 0.5X +MINUTE of timestamp 655 670 15 15.3 65.5 0.5X +SECOND of timestamp 757 763 7 13.2 75.7 0.4X +MILLISECONDS of timestamp 745 761 14 13.4 74.5 0.4X +MICROSECONDS of timestamp 691 697 7 14.5 69.1 0.5X +EPOCH of timestamp 794 806 12 12.6 79.4 0.4X Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 303 323 24 33.0 30.3 1.0X -MILLENNIUM of timestamp 841 852 9 11.9 84.1 0.4X -CENTURY of timestamp 818 821 4 12.2 81.8 0.4X -DECADE of timestamp 821 836 15 12.2 82.1 0.4X -YEAR of timestamp 858 928 61 11.7 85.8 0.4X -ISOYEAR of timestamp 933 939 6 10.7 93.3 0.3X -QUARTER of timestamp 979 1004 26 10.2 97.9 0.3X -MONTH of timestamp 755 784 25 13.2 75.5 0.4X -WEEK of timestamp 1189 1228 64 8.4 118.9 0.3X -DAY of timestamp 768 770 4 13.0 76.8 0.4X -DAYOFWEEK of timestamp 918 948 27 10.9 91.8 0.3X -DOW of timestamp 933 983 53 10.7 93.3 0.3X -ISODOW of timestamp 884 928 61 11.3 88.4 0.3X -DOY of timestamp 787 797 12 12.7 78.7 0.4X -HOUR of timestamp 611 664 58 16.4 61.1 0.5X -MINUTE of timestamp 616 622 5 16.2 61.6 0.5X -SECOND of timestamp 732 748 14 13.7 73.2 0.4X -MILLISECONDS of timestamp 705 716 17 14.2 70.5 0.4X -MICROSECONDS of timestamp 642 664 33 15.6 64.2 0.5X -EPOCH of timestamp 760 781 18 13.2 76.0 0.4X +cast to timestamp 273 274 1 36.6 27.3 1.0X +MILLENNIUM of timestamp 860 875 14 11.6 86.0 0.3X +CENTURY of timestamp 854 866 11 11.7 85.4 0.3X +DECADE of timestamp 855 863 10 11.7 85.5 0.3X +YEAR of timestamp 833 837 4 12.0 83.3 0.3X +ISOYEAR of timestamp 1022 1036 18 9.8 102.2 0.3X +QUARTER of timestamp 1067 1132 58 9.4 106.7 0.3X +MONTH of timestamp 855 861 5 11.7 85.5 0.3X +WEEK of timestamp 1312 1325 17 7.6 131.2 0.2X +DAY of timestamp 837 847 10 12.0 83.7 0.3X +DAYOFWEEK of timestamp 965 969 4 10.4 96.5 0.3X +DOW of timestamp 981 1038 56 10.2 98.1 0.3X +ISODOW of timestamp 942 959 20 10.6 94.2 0.3X +DOY of timestamp 994 1021 24 10.1 99.4 0.3X +HOUR of timestamp 690 695 7 14.5 69.0 0.4X +MINUTE of timestamp 764 788 21 13.1 76.4 0.4X +SECOND of timestamp 776 795 34 12.9 77.6 0.4X +MILLISECONDS of timestamp 863 879 14 11.6 86.3 0.3X +MICROSECONDS of timestamp 689 695 7 14.5 68.9 0.4X +EPOCH of timestamp 797 865 69 12.5 79.7 0.3X Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 712 786 120 14.0 71.2 1.0X -MILLENNIUM of date 846 888 73 11.8 84.6 0.8X -CENTURY of date 781 792 11 12.8 78.1 0.9X -DECADE of date 770 824 48 13.0 77.0 0.9X -YEAR of date 804 832 27 12.4 80.4 0.9X -ISOYEAR of date 909 931 24 11.0 90.9 0.8X -QUARTER of date 957 975 22 10.4 95.7 0.7X -MONTH of date 789 794 7 12.7 78.9 0.9X -WEEK of date 1141 1165 29 8.8 114.1 0.6X -DAY of date 784 800 22 12.8 78.4 0.9X -DAYOFWEEK of date 907 916 14 11.0 90.7 0.8X -DOW of date 931 958 25 10.7 93.1 0.8X -ISODOW of date 852 857 7 11.7 85.2 0.8X -DOY of date 831 870 45 12.0 83.1 0.9X -HOUR of date 1574 1598 26 6.4 157.4 0.5X -MINUTE of date 1525 1590 58 6.6 152.5 0.5X -SECOND of date 1728 1739 10 5.8 172.8 0.4X -MILLISECONDS of date 1715 1727 10 5.8 171.5 0.4X -MICROSECONDS of date 1559 1669 95 6.4 155.9 0.5X -EPOCH of date 1766 1774 9 5.7 176.6 0.4X +cast to date 763 774 13 13.1 76.3 1.0X +MILLENNIUM of date 857 911 73 11.7 85.7 0.9X +CENTURY of date 869 884 15 11.5 86.9 0.9X +DECADE of date 860 868 7 11.6 86.0 0.9X +YEAR of date 848 852 7 11.8 84.8 0.9X +ISOYEAR of date 1028 1046 22 9.7 102.8 0.7X +QUARTER of date 979 986 7 10.2 97.9 0.8X +MONTH of date 857 862 4 11.7 85.7 0.9X +WEEK of date 1218 1247 29 8.2 121.8 0.6X +DAY of date 855 868 11 11.7 85.5 0.9X +DAYOFWEEK of date 987 1081 82 10.1 98.7 0.8X +DOW of date 986 996 9 10.1 98.6 0.8X +ISODOW of date 938 947 10 10.7 93.8 0.8X +DOY of date 877 890 11 11.4 87.7 0.9X +HOUR of date 1656 1668 12 6.0 165.6 0.5X +MINUTE of date 1652 1666 15 6.1 165.2 0.5X +SECOND of date 1752 1776 23 5.7 175.2 0.4X +MILLISECONDS of date 1757 1766 11 5.7 175.7 0.4X +MICROSECONDS of date 1682 1691 14 5.9 168.2 0.5X +EPOCH of date 1706 1721 14 5.9 170.6 0.4X Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 803 825 23 12.5 80.3 1.0X -MILLENNIUM of date 918 924 8 10.9 91.8 0.9X -CENTURY of date 855 879 25 11.7 85.5 0.9X -DECADE of date 777 795 19 12.9 77.7 1.0X -YEAR of date 772 779 9 13.0 77.2 1.0X -ISOYEAR of date 903 927 21 11.1 90.3 0.9X -QUARTER of date 983 1339 381 10.2 98.3 0.8X -MONTH of date 764 780 20 13.1 76.4 1.1X -WEEK of date 1239 1316 112 8.1 123.9 0.6X -DAY of date 769 857 91 13.0 76.9 1.0X -DAYOFWEEK of date 900 965 61 11.1 90.0 0.9X -DOW of date 990 1041 62 10.1 99.0 0.8X -ISODOW of date 974 977 5 10.3 97.4 0.8X -DOY of date 862 918 50 11.6 86.2 0.9X -HOUR of date 1491 1517 25 6.7 149.1 0.5X -MINUTE of date 1514 1520 9 6.6 151.4 0.5X -SECOND of date 1657 1677 27 6.0 165.7 0.5X -MILLISECONDS of date 1646 1669 26 6.1 164.6 0.5X -MICROSECONDS of date 1538 1556 23 6.5 153.8 0.5X -EPOCH of date 1699 1718 28 5.9 169.9 0.5X +cast to date 771 779 10 13.0 77.1 1.0X +MILLENNIUM of date 839 858 22 11.9 83.9 0.9X +CENTURY of date 853 872 22 11.7 85.3 0.9X +DECADE of date 871 878 6 11.5 87.1 0.9X +YEAR of date 853 863 9 11.7 85.3 0.9X +ISOYEAR of date 1011 1021 11 9.9 101.1 0.8X +QUARTER of date 981 988 7 10.2 98.1 0.8X +MONTH of date 859 863 5 11.6 85.9 0.9X +WEEK of date 1148 1159 12 8.7 114.8 0.7X +DAY of date 852 855 2 11.7 85.2 0.9X +DAYOFWEEK of date 937 952 12 10.7 93.7 0.8X +DOW of date 953 956 3 10.5 95.3 0.8X +ISODOW of date 899 916 16 11.1 89.9 0.9X +DOY of date 857 865 7 11.7 85.7 0.9X +HOUR of date 1644 1662 18 6.1 164.4 0.5X +MINUTE of date 1633 1644 10 6.1 163.3 0.5X +SECOND of date 1719 1727 9 5.8 171.9 0.4X +MILLISECONDS of date 1721 1733 11 5.8 172.1 0.4X +MICROSECONDS of date 1656 1680 34 6.0 165.6 0.5X +EPOCH of date 1721 1736 15 5.8 172.1 0.4X + +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +Invoke extract for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to interval 939 947 9 10.7 93.9 1.0X +MILLENNIUM of interval 953 960 6 10.5 95.3 1.0X +CENTURY of interval 971 979 10 10.3 97.1 1.0X +DECADE of interval 962 969 7 10.4 96.2 1.0X +YEAR of interval 965 973 10 10.4 96.5 1.0X +QUARTER of interval 967 979 15 10.3 96.7 1.0X +MONTH of interval 951 979 26 10.5 95.1 1.0X +DAY of interval 945 970 25 10.6 94.5 1.0X +HOUR of interval 963 970 9 10.4 96.3 1.0X +MINUTE of interval 972 991 24 10.3 97.2 1.0X +SECOND of interval 1053 1057 8 9.5 105.3 0.9X +MILLISECONDS of interval 1042 1048 11 9.6 104.2 0.9X +MICROSECONDS of interval 963 967 4 10.4 96.3 1.0X +EPOCH of interval 1051 1071 22 9.5 105.1 0.9X Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to interval 971 976 5 10.3 97.1 1.0X -MILLENNIUM of interval 983 1009 23 10.2 98.3 1.0X -CENTURY of interval 972 984 12 10.3 97.2 1.0X -DECADE of interval 954 962 10 10.5 95.4 1.0X -YEAR of interval 954 973 17 10.5 95.4 1.0X -QUARTER of interval 1009 1020 11 9.9 100.9 1.0X -MONTH of interval 946 963 16 10.6 94.6 1.0X -DAY of interval 952 963 12 10.5 95.2 1.0X -HOUR of interval 948 960 11 10.5 94.8 1.0X -MINUTE of interval 1035 1040 4 9.7 103.5 0.9X -SECOND of interval 1085 1105 31 9.2 108.5 0.9X -MILLISECONDS of interval 1065 1088 24 9.4 106.5 0.9X -MICROSECONDS of interval 992 1007 17 10.1 99.2 1.0X -EPOCH of interval 1087 1103 25 9.2 108.7 0.9X +cast to interval 944 952 8 10.6 94.4 1.0X +MILLENNIUM of interval 955 960 5 10.5 95.5 1.0X +CENTURY of interval 958 972 12 10.4 95.8 1.0X +DECADE of interval 962 967 6 10.4 96.2 1.0X +YEAR of interval 958 978 18 10.4 95.8 1.0X +QUARTER of interval 979 998 19 10.2 97.9 1.0X +MONTH of interval 949 963 12 10.5 94.9 1.0X +DAY of interval 958 971 11 10.4 95.8 1.0X +HOUR of interval 947 962 15 10.6 94.7 1.0X +MINUTE of interval 989 1006 15 10.1 98.9 1.0X +SECOND of interval 1061 1064 4 9.4 106.1 0.9X +MILLISECONDS of interval 1042 1061 18 9.6 104.2 0.9X +MICROSECONDS of interval 982 992 12 10.2 98.2 1.0X +EPOCH of interval 1047 1057 13 9.5 104.7 0.9X diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 36bdbd026c046..94cf0bc475582 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -2,118 +2,137 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 287 308 19 34.8 28.7 1.0X -MILLENNIUM of timestamp 896 918 20 11.2 89.6 0.3X -CENTURY of timestamp 849 856 7 11.8 84.9 0.3X -DECADE of timestamp 765 777 17 13.1 76.5 0.4X -YEAR of timestamp 754 756 2 13.3 75.4 0.4X -ISOYEAR of timestamp 843 849 5 11.9 84.3 0.3X -QUARTER of timestamp 867 873 9 11.5 86.7 0.3X -MONTH of timestamp 758 762 4 13.2 75.8 0.4X -WEEK of timestamp 1049 1054 6 9.5 104.9 0.3X -DAY of timestamp 750 763 11 13.3 75.0 0.4X -DAYOFWEEK of timestamp 890 918 25 11.2 89.0 0.3X -DOW of timestamp 879 887 8 11.4 87.9 0.3X -ISODOW of timestamp 862 869 11 11.6 86.2 0.3X -DOY of timestamp 811 868 55 12.3 81.1 0.4X -HOUR of timestamp 627 638 11 16.0 62.7 0.5X -MINUTE of timestamp 600 606 6 16.7 60.0 0.5X -SECOND of timestamp 743 799 51 13.5 74.3 0.4X -MILLISECONDS of timestamp 723 737 22 13.8 72.3 0.4X -MICROSECONDS of timestamp 648 653 5 15.4 64.8 0.4X -EPOCH of timestamp 780 800 17 12.8 78.0 0.4X +cast to timestamp 275 307 28 36.3 27.5 1.0X +MILLENNIUM of timestamp 911 924 20 11.0 91.1 0.3X +CENTURY of timestamp 846 850 5 11.8 84.6 0.3X +DECADE of timestamp 744 761 21 13.4 74.4 0.4X +YEAR of timestamp 750 764 15 13.3 75.0 0.4X +ISOYEAR of timestamp 815 828 11 12.3 81.5 0.3X +QUARTER of timestamp 882 895 12 11.3 88.2 0.3X +MONTH of timestamp 739 760 19 13.5 73.9 0.4X +WEEK of timestamp 1058 1082 23 9.4 105.8 0.3X +DAY of timestamp 722 730 8 13.8 72.2 0.4X +DAYOFWEEK of timestamp 860 907 58 11.6 86.0 0.3X +DOW of timestamp 853 860 7 11.7 85.3 0.3X +ISODOW of timestamp 829 835 8 12.1 82.9 0.3X +DOY of timestamp 757 772 20 13.2 75.7 0.4X +HOUR of timestamp 586 594 7 17.1 58.6 0.5X +MINUTE of timestamp 577 584 8 17.3 57.7 0.5X +SECOND of timestamp 810 827 17 12.4 81.0 0.3X +MILLISECONDS of timestamp 687 704 16 14.6 68.7 0.4X +MICROSECONDS of timestamp 628 632 6 15.9 62.8 0.4X +EPOCH of timestamp 750 761 12 13.3 75.0 0.4X Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 238 248 12 42.0 23.8 1.0X -MILLENNIUM of timestamp 862 875 12 11.6 86.2 0.3X -CENTURY of timestamp 833 847 22 12.0 83.3 0.3X -DECADE of timestamp 759 765 7 13.2 75.9 0.3X -YEAR of timestamp 744 755 15 13.4 74.4 0.3X -ISOYEAR of timestamp 937 1019 73 10.7 93.7 0.3X -QUARTER of timestamp 1011 1091 69 9.9 101.1 0.2X -MONTH of timestamp 846 888 40 11.8 84.6 0.3X -WEEK of timestamp 1210 1239 41 8.3 121.0 0.2X -DAY of timestamp 932 979 41 10.7 93.2 0.3X -DAYOFWEEK of timestamp 1174 1200 42 8.5 117.4 0.2X -DOW of timestamp 1131 1172 37 8.8 113.1 0.2X -ISODOW of timestamp 896 903 7 11.2 89.6 0.3X -DOY of timestamp 805 818 12 12.4 80.5 0.3X -HOUR of timestamp 596 597 2 16.8 59.6 0.4X -MINUTE of timestamp 582 597 17 17.2 58.2 0.4X -SECOND of timestamp 697 709 16 14.4 69.7 0.3X -MILLISECONDS of timestamp 700 710 10 14.3 70.0 0.3X -MICROSECONDS of timestamp 612 631 21 16.3 61.2 0.4X -EPOCH of timestamp 755 760 7 13.2 75.5 0.3X +cast to timestamp 238 242 4 42.0 23.8 1.0X +MILLENNIUM of timestamp 811 835 25 12.3 81.1 0.3X +CENTURY of timestamp 799 823 21 12.5 79.9 0.3X +DECADE of timestamp 733 743 12 13.6 73.3 0.3X +YEAR of timestamp 714 717 3 14.0 71.4 0.3X +ISOYEAR of timestamp 874 900 24 11.4 87.4 0.3X +QUARTER of timestamp 903 910 11 11.1 90.3 0.3X +MONTH of timestamp 774 808 44 12.9 77.4 0.3X +WEEK of timestamp 1053 1064 12 9.5 105.3 0.2X +DAY of timestamp 755 770 14 13.2 75.5 0.3X +DAYOFWEEK of timestamp 831 841 9 12.0 83.1 0.3X +DOW of timestamp 871 898 23 11.5 87.1 0.3X +ISODOW of timestamp 832 887 59 12.0 83.2 0.3X +DOY of timestamp 755 767 13 13.2 75.5 0.3X +HOUR of timestamp 580 597 27 17.3 58.0 0.4X +MINUTE of timestamp 572 589 15 17.5 57.2 0.4X +SECOND of timestamp 716 730 12 14.0 71.6 0.3X +MILLISECONDS of timestamp 716 762 57 14.0 71.6 0.3X +MICROSECONDS of timestamp 600 610 12 16.7 60.0 0.4X +EPOCH of timestamp 751 779 44 13.3 75.1 0.3X Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 633 641 7 15.8 63.3 1.0X -MILLENNIUM of date 824 845 26 12.1 82.4 0.8X -CENTURY of date 864 878 13 11.6 86.4 0.7X -DECADE of date 746 763 17 13.4 74.6 0.8X -YEAR of date 752 785 39 13.3 75.2 0.8X -ISOYEAR of date 900 905 5 11.1 90.0 0.7X -QUARTER of date 906 930 23 11.0 90.6 0.7X -MONTH of date 747 752 6 13.4 74.7 0.8X -WEEK of date 1089 1100 17 9.2 108.9 0.6X -DAY of date 795 803 13 12.6 79.5 0.8X -DAYOFWEEK of date 937 944 10 10.7 93.7 0.7X -DOW of date 927 1003 66 10.8 92.7 0.7X -ISODOW of date 977 980 7 10.2 97.7 0.6X -DOY of date 827 877 45 12.1 82.7 0.8X -HOUR of date 1525 1547 34 6.6 152.5 0.4X -MINUTE of date 1473 1499 23 6.8 147.3 0.4X -SECOND of date 1600 1615 15 6.2 160.0 0.4X -MILLISECONDS of date 1666 1765 156 6.0 166.6 0.4X -MICROSECONDS of date 1554 1627 127 6.4 155.4 0.4X -EPOCH of date 1615 1646 27 6.2 161.5 0.4X +cast to date 629 634 6 15.9 62.9 1.0X +MILLENNIUM of date 826 912 97 12.1 82.6 0.8X +CENTURY of date 797 840 54 12.6 79.7 0.8X +DECADE of date 734 737 4 13.6 73.4 0.9X +YEAR of date 721 768 78 13.9 72.1 0.9X +ISOYEAR of date 884 892 13 11.3 88.4 0.7X +QUARTER of date 894 949 69 11.2 89.4 0.7X +MONTH of date 726 729 2 13.8 72.6 0.9X +WEEK of date 1013 1023 9 9.9 101.3 0.6X +DAY of date 727 747 26 13.8 72.7 0.9X +DAYOFWEEK of date 828 832 6 12.1 82.8 0.8X +DOW of date 846 850 5 11.8 84.6 0.7X +ISODOW of date 803 810 7 12.5 80.3 0.8X +DOY of date 751 761 11 13.3 75.1 0.8X +HOUR of date 1372 1376 5 7.3 137.2 0.5X +MINUTE of date 1379 1398 20 7.3 137.9 0.5X +SECOND of date 1530 1542 10 6.5 153.0 0.4X +MILLISECONDS of date 1532 1541 8 6.5 153.2 0.4X +MICROSECONDS of date 1442 1462 23 6.9 144.2 0.4X +EPOCH of date 1573 1594 20 6.4 157.3 0.4X Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 676 690 20 14.8 67.6 1.0X -MILLENNIUM of date 868 882 16 11.5 86.8 0.8X -CENTURY of date 875 880 8 11.4 87.5 0.8X -DECADE of date 762 776 16 13.1 76.2 0.9X -YEAR of date 788 800 11 12.7 78.8 0.9X -ISOYEAR of date 903 917 12 11.1 90.3 0.7X -QUARTER of date 983 1018 40 10.2 98.3 0.7X -MONTH of date 836 857 19 12.0 83.6 0.8X -WEEK of date 1137 1168 28 8.8 113.7 0.6X -DAY of date 768 817 82 13.0 76.8 0.9X -DAYOFWEEK of date 890 926 36 11.2 89.0 0.8X -DOW of date 1007 1033 39 9.9 100.7 0.7X -ISODOW of date 962 969 7 10.4 96.2 0.7X -DOY of date 797 882 80 12.6 79.7 0.8X -HOUR of date 1449 1482 29 6.9 144.9 0.5X -MINUTE of date 1536 1610 69 6.5 153.6 0.4X -SECOND of date 1675 1823 128 6.0 167.5 0.4X -MILLISECONDS of date 1605 1622 18 6.2 160.5 0.4X -MICROSECONDS of date 1481 1504 32 6.8 148.1 0.5X -EPOCH of date 1656 1843 296 6.0 165.6 0.4X +cast to date 622 633 14 16.1 62.2 1.0X +MILLENNIUM of date 793 814 20 12.6 79.3 0.8X +CENTURY of date 798 809 12 12.5 79.8 0.8X +DECADE of date 735 740 6 13.6 73.5 0.8X +YEAR of date 714 732 23 14.0 71.4 0.9X +ISOYEAR of date 881 886 4 11.3 88.1 0.7X +QUARTER of date 916 925 14 10.9 91.6 0.7X +MONTH of date 732 737 5 13.7 73.2 0.8X +WEEK of date 1022 1034 10 9.8 102.2 0.6X +DAY of date 722 745 26 13.8 72.2 0.9X +DAYOFWEEK of date 831 849 17 12.0 83.1 0.7X +DOW of date 853 858 4 11.7 85.3 0.7X +ISODOW of date 824 825 1 12.1 82.4 0.8X +DOY of date 749 752 4 13.3 74.9 0.8X +HOUR of date 1422 1425 3 7.0 142.2 0.4X +MINUTE of date 1379 1394 13 7.3 137.9 0.5X +SECOND of date 1525 1536 9 6.6 152.5 0.4X +MILLISECONDS of date 1542 1555 14 6.5 154.2 0.4X +MICROSECONDS of date 1446 1449 2 6.9 144.6 0.4X +EPOCH of date 1589 1621 35 6.3 158.9 0.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +Invoke extract for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to interval 913 921 8 11.0 91.3 1.0X +MILLENNIUM of interval 976 983 12 10.2 97.6 0.9X +CENTURY of interval 976 979 3 10.2 97.6 0.9X +DECADE of interval 975 987 10 10.3 97.5 0.9X +YEAR of interval 964 968 5 10.4 96.4 0.9X +QUARTER of interval 987 997 12 10.1 98.7 0.9X +MONTH of interval 974 983 11 10.3 97.4 0.9X +DAY of interval 959 962 3 10.4 95.9 1.0X +HOUR of interval 972 986 13 10.3 97.2 0.9X +MINUTE of interval 985 988 4 10.2 98.5 0.9X +SECOND of interval 1128 1141 13 8.9 112.8 0.8X +MILLISECONDS of interval 1083 1086 3 9.2 108.3 0.8X +MICROSECONDS of interval 963 969 5 10.4 96.3 0.9X +EPOCH of interval 1094 1116 23 9.1 109.4 0.8X Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to interval 919 959 45 10.9 91.9 1.0X -MILLENNIUM of interval 959 1001 36 10.4 95.9 1.0X -CENTURY of interval 974 998 21 10.3 97.4 0.9X -DECADE of interval 975 989 13 10.3 97.5 0.9X -YEAR of interval 987 1012 24 10.1 98.7 0.9X -QUARTER of interval 1014 1031 25 9.9 101.4 0.9X -MONTH of interval 978 1018 45 10.2 97.8 0.9X -DAY of interval 1012 1095 92 9.9 101.2 0.9X -HOUR of interval 1129 1143 14 8.9 112.9 0.8X -MINUTE of interval 1038 1060 25 9.6 103.8 0.9X -SECOND of interval 1062 1289 356 9.4 106.2 0.9X -MILLISECONDS of interval 1142 1226 83 8.8 114.2 0.8X -MICROSECONDS of interval 1038 1129 79 9.6 103.8 0.9X -EPOCH of interval 1090 1127 34 9.2 109.0 0.8X +cast to interval 950 971 23 10.5 95.0 1.0X +MILLENNIUM of interval 989 1000 11 10.1 98.9 1.0X +CENTURY of interval 984 990 6 10.2 98.4 1.0X +DECADE of interval 962 978 14 10.4 96.2 1.0X +YEAR of interval 945 963 16 10.6 94.5 1.0X +QUARTER of interval 985 998 17 10.2 98.5 1.0X +MONTH of interval 970 976 9 10.3 97.0 1.0X +DAY of interval 953 962 8 10.5 95.3 1.0X +HOUR of interval 942 956 17 10.6 94.2 1.0X +MINUTE of interval 975 993 16 10.3 97.5 1.0X +SECOND of interval 1110 1122 12 9.0 111.0 0.9X +MILLISECONDS of interval 1056 1074 22 9.5 105.6 0.9X +MICROSECONDS of interval 939 960 36 10.7 93.9 1.0X +EPOCH of interval 1071 1093 21 9.3 107.1 0.9X diff --git a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt index 920e0a7723e70..03bc334471e56 100644 --- a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt @@ -3,110 +3,110 @@ Benchmark for performance of JSON parsing ================================================================================================ Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 84774 84927 264 1.2 847.7 1.0X -UTF-8 is set 119081 120155 1773 0.8 1190.8 0.7X +No encoding 46010 46118 113 2.2 460.1 1.0X +UTF-8 is set 54407 55427 1718 1.8 544.1 0.8X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 49293 49356 70 2.0 492.9 1.0X -UTF-8 is set 80183 80211 25 1.2 801.8 0.6X +No encoding 26614 28220 1461 3.8 266.1 1.0X +UTF-8 is set 42765 43400 550 2.3 427.6 0.6X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 61070 61476 536 0.2 6107.0 1.0X -UTF-8 is set 109765 109881 102 0.1 10976.5 0.6X +No encoding 35696 35821 113 0.3 3569.6 1.0X +UTF-8 is set 55441 56176 1037 0.2 5544.1 0.6X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 176999 178163 1008 0.0 353997.9 1.0X -UTF-8 is set 201209 201641 614 0.0 402419.0 0.9X +No encoding 61514 62968 NaN 0.0 123027.2 1.0X +UTF-8 is set 72096 72933 1162 0.0 144192.7 0.9X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns 18768 20587 496 0.5 1876.8 1.0X -Select 1 column 22642 22644 3 0.4 2264.2 0.8X +Select 10 columns 9859 9913 79 1.0 985.9 1.0X +Select 1 column 10981 11003 36 0.9 1098.1 0.9X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Short column without encoding 7697 7738 55 1.3 769.7 1.0X -Short column with UTF-8 14051 14189 176 0.7 1405.1 0.5X -Wide column without encoding 108999 110075 1085 0.1 10899.9 0.1X -Wide column with UTF-8 157433 157779 308 0.1 15743.3 0.0X +Short column without encoding 3555 3579 27 2.8 355.5 1.0X +Short column with UTF-8 5204 5227 35 1.9 520.4 0.7X +Wide column without encoding 60458 60637 164 0.2 6045.8 0.1X +Wide column with UTF-8 77544 78111 551 0.1 7754.4 0.0X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 644 647 4 15.5 64.4 1.0X -from_json 25859 25872 12 0.4 2585.9 0.0X -json_tuple 31679 31761 71 0.3 3167.9 0.0X -get_json_object 24772 25220 389 0.4 2477.2 0.0X +Text read 342 346 3 29.2 34.2 1.0X +from_json 7123 7318 179 1.4 712.3 0.0X +json_tuple 9843 9957 132 1.0 984.3 0.0X +get_json_object 7827 8046 194 1.3 782.7 0.0X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 3135 3165 52 15.9 62.7 1.0X -schema inferring 29383 29389 10 1.7 587.7 0.1X -parsing 32623 35183 NaN 1.5 652.5 0.1X +Text read 1856 1884 32 26.9 37.1 1.0X +schema inferring 16734 16900 153 3.0 334.7 0.1X +parsing 14884 15203 470 3.4 297.7 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 11874 11948 82 4.2 237.5 1.0X -Schema inferring 42382 42398 23 1.2 847.6 0.3X -Parsing without charset 36410 36442 54 1.4 728.2 0.3X -Parsing with UTF-8 62412 62463 48 0.8 1248.2 0.2X +Text read 5932 6148 228 8.4 118.6 1.0X +Schema inferring 20836 21938 1086 2.4 416.7 0.3X +Parsing without charset 18134 18661 457 2.8 362.7 0.3X +Parsing with UTF-8 27734 28069 378 1.8 554.7 0.2X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2191 2209 20 4.6 219.1 1.0X -to_json(timestamp) 18670 19042 565 0.5 1867.0 0.1X -write timestamps to files 11836 13156 NaN 0.8 1183.6 0.2X -Create a dataset of dates 2321 2351 33 4.3 232.1 0.9X -to_json(date) 12703 12726 24 0.8 1270.3 0.2X -write dates to files 8230 8303 76 1.2 823.0 0.3X +Create a dataset of timestamps 889 914 28 11.2 88.9 1.0X +to_json(timestamp) 7920 8172 353 1.3 792.0 0.1X +write timestamps to files 6726 6822 129 1.5 672.6 0.1X +Create a dataset of dates 953 963 12 10.5 95.3 0.9X +to_json(date) 5370 5705 320 1.9 537.0 0.2X +write dates to files 4109 4166 52 2.4 410.9 0.2X -OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2780 2795 13 3.6 278.0 1.0X -read timestamps from files 37158 37305 137 0.3 3715.8 0.1X -infer timestamps from files 73666 73838 149 0.1 7366.6 0.0X -read date text from files 2597 2609 10 3.9 259.7 1.1X -read date from files 24439 24501 56 0.4 2443.9 0.1X -timestamp strings 3052 3064 12 3.3 305.2 0.9X -parse timestamps from Dataset[String] 43611 43665 52 0.2 4361.1 0.1X -infer timestamps from Dataset[String] 83745 84153 376 0.1 8374.5 0.0X -date strings 4068 4076 10 2.5 406.8 0.7X -parse dates from Dataset[String] 34700 34807 118 0.3 3470.0 0.1X -from_json(timestamp) 64074 64124 53 0.2 6407.4 0.0X -from_json(date) 52520 52617 101 0.2 5252.0 0.1X +read timestamp text from files 1614 1675 55 6.2 161.4 1.0X +read timestamps from files 16640 16858 209 0.6 1664.0 0.1X +infer timestamps from files 33239 33388 227 0.3 3323.9 0.0X +read date text from files 1310 1340 44 7.6 131.0 1.2X +read date from files 9470 9513 41 1.1 947.0 0.2X +timestamp strings 1303 1342 47 7.7 130.3 1.2X +parse timestamps from Dataset[String] 17650 18073 380 0.6 1765.0 0.1X +infer timestamps from Dataset[String] 32623 34065 1330 0.3 3262.3 0.0X +date strings 1864 1871 7 5.4 186.4 0.9X +parse dates from Dataset[String] 10914 11316 482 0.9 1091.4 0.1X +from_json(timestamp) 21102 21990 929 0.5 2110.2 0.1X +from_json(date) 15275 15961 598 0.7 1527.5 0.1X diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt index e435f573a5d92..0f188c4cdea56 100644 --- a/sql/core/benchmarks/JsonBenchmark-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-results.txt @@ -3,110 +3,110 @@ Benchmark for performance of JSON parsing ================================================================================================ Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 61888 61918 27 1.6 618.9 1.0X -UTF-8 is set 109057 113663 NaN 0.9 1090.6 0.6X +No encoding 38998 41002 NaN 2.6 390.0 1.0X +UTF-8 is set 61231 63282 1854 1.6 612.3 0.6X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 44517 44535 29 2.2 445.2 1.0X -UTF-8 is set 75722 75840 111 1.3 757.2 0.6X +No encoding 28272 28338 70 3.5 282.7 1.0X +UTF-8 is set 58681 62243 1517 1.7 586.8 0.5X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 63677 64090 633 0.2 6367.7 1.0X -UTF-8 is set 99424 99615 185 0.1 9942.4 0.6X +No encoding 44026 51829 1329 0.2 4402.6 1.0X +UTF-8 is set 65839 68596 500 0.2 6583.9 0.7X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 174052 174251 174 0.0 348104.1 1.0X -UTF-8 is set 189000 189098 113 0.0 378000.9 0.9X +No encoding 72144 74820 NaN 0.0 144287.6 1.0X +UTF-8 is set 69571 77888 NaN 0.0 139142.3 1.0X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns 18387 18473 142 0.5 1838.7 1.0X -Select 1 column 25560 25571 13 0.4 2556.0 0.7X +Select 10 columns 9502 9604 106 1.1 950.2 1.0X +Select 1 column 11861 11948 109 0.8 1186.1 0.8X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Short column without encoding 9323 9384 58 1.1 932.3 1.0X -Short column with UTF-8 14016 14058 55 0.7 1401.6 0.7X -Wide column without encoding 133258 133532 382 0.1 13325.8 0.1X -Wide column with UTF-8 181212 181283 61 0.1 18121.2 0.1X +Short column without encoding 3830 3846 15 2.6 383.0 1.0X +Short column with UTF-8 5538 5543 7 1.8 553.8 0.7X +Wide column without encoding 66899 69158 NaN 0.1 6689.9 0.1X +Wide column with UTF-8 90052 93235 NaN 0.1 9005.2 0.0X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 1168 1174 5 8.6 116.8 1.0X -from_json 22604 23571 883 0.4 2260.4 0.1X -json_tuple 29979 30053 91 0.3 2997.9 0.0X -get_json_object 21987 22263 241 0.5 2198.7 0.1X +Text read 659 674 13 15.2 65.9 1.0X +from_json 7676 7943 405 1.3 767.6 0.1X +json_tuple 9881 10172 273 1.0 988.1 0.1X +get_json_object 7949 8055 119 1.3 794.9 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 5831 5842 14 8.6 116.6 1.0X -schema inferring 31372 31456 73 1.6 627.4 0.2X -parsing 35911 36191 254 1.4 718.2 0.2X +Text read 3314 3326 17 15.1 66.3 1.0X +schema inferring 16549 17037 484 3.0 331.0 0.2X +parsing 15138 15283 172 3.3 302.8 0.2X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 10249 10314 77 4.9 205.0 1.0X -Schema inferring 35403 35436 40 1.4 708.1 0.3X -Parsing without charset 32875 32879 4 1.5 657.5 0.3X -Parsing with UTF-8 53444 53519 100 0.9 1068.9 0.2X +Text read 5136 5446 268 9.7 102.7 1.0X +Schema inferring 19864 20568 1191 2.5 397.3 0.3X +Parsing without charset 17535 17888 329 2.9 350.7 0.3X +Parsing with UTF-8 25609 25758 218 2.0 512.2 0.2X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 1909 1924 17 5.2 190.9 1.0X -to_json(timestamp) 18956 19122 208 0.5 1895.6 0.1X -write timestamps to files 13446 13472 43 0.7 1344.6 0.1X -Create a dataset of dates 2180 2200 28 4.6 218.0 0.9X -to_json(date) 12780 12899 109 0.8 1278.0 0.1X -write dates to files 7835 7865 29 1.3 783.5 0.2X +Create a dataset of timestamps 784 790 7 12.8 78.4 1.0X +to_json(timestamp) 8005 8055 50 1.2 800.5 0.1X +write timestamps to files 6515 6559 45 1.5 651.5 0.1X +Create a dataset of dates 854 881 24 11.7 85.4 0.9X +to_json(date) 5187 5194 7 1.9 518.7 0.2X +write dates to files 3663 3684 22 2.7 366.3 0.2X -OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2467 2477 9 4.1 246.7 1.0X -read timestamps from files 40186 40342 135 0.2 4018.6 0.1X -infer timestamps from files 82005 82079 71 0.1 8200.5 0.0X -read date text from files 2243 2264 22 4.5 224.3 1.1X -read date from files 24852 24863 19 0.4 2485.2 0.1X -timestamp strings 3836 3854 16 2.6 383.6 0.6X -parse timestamps from Dataset[String] 51521 51697 242 0.2 5152.1 0.0X -infer timestamps from Dataset[String] 97300 97398 133 0.1 9730.0 0.0X -date strings 4488 4491 5 2.2 448.8 0.5X -parse dates from Dataset[String] 37918 37976 68 0.3 3791.8 0.1X -from_json(timestamp) 69611 69632 36 0.1 6961.1 0.0X -from_json(date) 56598 56974 347 0.2 5659.8 0.0X +read timestamp text from files 1297 1316 26 7.7 129.7 1.0X +read timestamps from files 16915 17723 963 0.6 1691.5 0.1X +infer timestamps from files 33967 34304 360 0.3 3396.7 0.0X +read date text from files 1095 1100 7 9.1 109.5 1.2X +read date from files 8376 8513 209 1.2 837.6 0.2X +timestamp strings 1807 1816 8 5.5 180.7 0.7X +parse timestamps from Dataset[String] 18189 18242 74 0.5 1818.9 0.1X +infer timestamps from Dataset[String] 37906 38547 571 0.3 3790.6 0.0X +date strings 2191 2194 4 4.6 219.1 0.6X +parse dates from Dataset[String] 11593 11625 33 0.9 1159.3 0.1X +from_json(timestamp) 22589 22650 101 0.4 2258.9 0.1X +from_json(date) 16479 16619 159 0.6 1647.9 0.1X diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c95fe3ce1c120..e97c7fd3280be 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -131,8 +131,8 @@ test - mysql - mysql-connector-java + org.mariadb.jdbc + mariadb-java-client test diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 6fccb629a3811..c50619a4c12e3 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -35,8 +35,10 @@ import org.apache.parquet.schema.PrimitiveType; import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.catalyst.util.RebaseDateTime; import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException; import org.apache.spark.sql.execution.vectorized.WritableColumnVector; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DecimalType; @@ -101,6 +103,7 @@ public class VectorizedColumnReader { // The timezone conversion to apply to int96 timestamps. Null if no conversion. private final ZoneId convertTz; private static final ZoneId UTC = ZoneOffset.UTC; + private final boolean rebaseDateTime; public VectorizedColumnReader( ColumnDescriptor descriptor, @@ -129,6 +132,7 @@ public VectorizedColumnReader( if (totalValueCount == 0) { throw new IOException("totalValueCount == 0"); } + this.rebaseDateTime = SQLConf.get().parquetRebaseDateTimeInReadEnabled(); } /** @@ -407,7 +411,7 @@ private void readBooleanBatch(int rowId, int num, WritableColumnVector column) private void readIntBatch(int rowId, int num, WritableColumnVector column) throws IOException { // This is where we implement support for the valid type conversions. // TODO: implement remaining type conversions - if (column.dataType() == DataTypes.IntegerType || column.dataType() == DataTypes.DateType || + if (column.dataType() == DataTypes.IntegerType || DecimalType.is32BitDecimalType(column.dataType())) { defColumn.readIntegers( num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); @@ -417,6 +421,21 @@ private void readIntBatch(int rowId, int num, WritableColumnVector column) throw } else if (column.dataType() == DataTypes.ShortType) { defColumn.readShorts( num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else if (column.dataType() == DataTypes.DateType ) { + if (rebaseDateTime) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putInt( + rowId + i, + RebaseDateTime.rebaseJulianToGregorianDays(dataColumn.readInteger())); + } else { + column.putNull(rowId + i); + } + } + } else { + defColumn.readIntegers( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } } else { throw constructConvertNotSupportedException(descriptor, column); } @@ -425,16 +444,41 @@ private void readIntBatch(int rowId, int num, WritableColumnVector column) throw private void readLongBatch(int rowId, int num, WritableColumnVector column) throws IOException { // This is where we implement support for the valid type conversions. if (column.dataType() == DataTypes.LongType || - DecimalType.is64BitDecimalType(column.dataType()) || - originalType == OriginalType.TIMESTAMP_MICROS) { + DecimalType.is64BitDecimalType(column.dataType())) { defColumn.readLongs( num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else if (originalType == OriginalType.TIMESTAMP_MICROS) { + if (rebaseDateTime) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putLong( + rowId + i, + RebaseDateTime.rebaseJulianToGregorianMicros(dataColumn.readLong())); + } else { + column.putNull(rowId + i); + } + } + } else { + defColumn.readLongs( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } } else if (originalType == OriginalType.TIMESTAMP_MILLIS) { - for (int i = 0; i < num; i++) { - if (defColumn.readInteger() == maxDefLevel) { - column.putLong(rowId + i, DateTimeUtils.millisToMicros(dataColumn.readLong())); - } else { - column.putNull(rowId + i); + if (rebaseDateTime) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + long micros = DateTimeUtils.millisToMicros(dataColumn.readLong()); + column.putLong(rowId + i, RebaseDateTime.rebaseJulianToGregorianMicros(micros)); + } else { + column.putNull(rowId + i); + } + } + } else { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putLong(rowId + i, DateTimeUtils.millisToMicros(dataColumn.readLong())); + } else { + column.putNull(rowId + i); + } } } } else { diff --git a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css index 94a6bd85e774c..6ba79dc74718a 100644 --- a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css +++ b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css @@ -35,7 +35,7 @@ } /* Highlight the SparkPlan node name */ -#plan-viz-graph svg text :first-child { +#plan-viz-graph svg text :first-child:not(.stageId-and-taskId-metrics) { font-weight: bold; } diff --git a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js index 6244b2ccdba15..bb393d9f2c638 100644 --- a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js +++ b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js @@ -47,6 +47,7 @@ function renderPlanViz() { } resizeSvg(svg); + postprocessForAdditionalMetrics(); } /* -------------------- * @@ -70,6 +71,10 @@ function setupTooltipForSparkPlanNode(nodeId) { }) } +// labelSeparator should be a non-graphical character in order not to affect the width of boxes. +var labelSeparator = "\x01"; +var stageAndTaskMetricsPattern = "^(.*)(\\(stage.*task[^)]*\\))(.*)$"; + /* * Helper function to pre-process the graph layout. * This step is necessary for certain styles that affect the positioning @@ -79,8 +84,29 @@ function preprocessGraphLayout(g) { g.graph().ranksep = "70"; var nodes = g.nodes(); for (var i = 0; i < nodes.length; i++) { - var node = g.node(nodes[i]); - node.padding = "5"; + var node = g.node(nodes[i]); + node.padding = "5"; + + var firstSearator; + var secondSeparator; + var splitter; + if (node.isCluster) { + firstSearator = secondSeparator = labelSeparator; + splitter = "\\n"; + } else { + firstSearator = ""; + secondSeparator = ""; + splitter = "
    "; + } + + node.label.split(splitter).forEach(function(text, i) { + var newTexts = text.match(stageAndTaskMetricsPattern); + if (newTexts) { + node.label = node.label.replace( + newTexts[0], + newTexts[1] + firstSearator + newTexts[2] + secondSeparator + newTexts[3]); + } + }); } // Curve the edges var edges = g.edges(); @@ -158,3 +184,47 @@ function getAbsolutePosition(d3selection) { } return { x: _x, y: _y }; } + +/* + * Helper function for postprocess for additional metrics. + */ +function postprocessForAdditionalMetrics() { + // With dagre-d3, we can choose normal text (default) or HTML as a label type. + // HTML label for node works well but not for cluster so we need to choose the default label type + // and manipulate DOM. + $("g.cluster text tspan") + .each(function() { + var originalText = $(this).text(); + if (originalText.indexOf(labelSeparator) > 0) { + var newTexts = originalText.split(labelSeparator); + var thisD3Node = d3.selectAll($(this)); + thisD3Node.text(newTexts[0]); + thisD3Node.append("tspan").attr("class", "stageId-and-taskId-metrics").text(newTexts[1]); + $(this).append(newTexts[2]); + } else { + return originalText; + } + }); + + var checkboxNode = $("#stageId-and-taskId-checkbox"); + checkboxNode.click(function() { + onClickAdditionalMetricsCheckbox($(this)); + }); + var isChecked = window.localStorage.getItem("stageId-and-taskId-checked") == "true"; + $("#stageId-and-taskId-checkbox").prop("checked", isChecked); + onClickAdditionalMetricsCheckbox(checkboxNode); +} + +/* + * Helper function which defines the action on click the checkbox. + */ +function onClickAdditionalMetricsCheckbox(checkboxNode) { + var additionalMetrics = $(".stageId-and-taskId-metrics"); + var isChecked = checkboxNode.prop("checked"); + if (isChecked) { + additionalMetrics.show(); + } else { + additionalMetrics.hide(); + } + window.localStorage.setItem("stageId-and-taskId-checked", isChecked); +} 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 8bd5835fd931b..49c9f830fb27e 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 @@ -319,6 +319,24 @@ class Column(val expr: Expression) extends Logging { */ def =!= (other: Any): Column = withExpr{ Not(EqualTo(expr, lit(other).expr)) } + /** + * Inequality test. + * {{{ + * // Scala: + * df.select( df("colA") !== df("colB") ) + * df.select( !(df("colA") === df("colB")) ) + * + * // Java: + * import static org.apache.spark.sql.functions.*; + * df.filter( col("colA").notEqual(col("colB")) ); + * }}} + * + * @group expr_ops + * @since 1.3.0 + */ + @deprecated("!== does not have the same precedence as ===, use =!= instead", "2.0.0") + def !== (other: Any): Column = this =!= other + /** * Inequality test. * {{{ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 9416126780506..a7b3d08ec4df7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -96,8 +96,19 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to parse timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to parse timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 1.4.0 @@ -133,8 +144,19 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to parse timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to parse timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 1.4.0 @@ -149,8 +171,19 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to parse timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to parse timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 1.4.0 @@ -391,11 +424,15 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * Custom date formats follow the formats at + * + * Datetime Patterns. * This applies to date type.
  • - *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.time.format.DateTimeFormatter`. This applies to timestamp type.
  • + * + * Datetime Patterns. + * This applies to timestamp type. *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • *
  • `encoding` (by default it is not set): allows to forcibly set one of standard basic @@ -616,11 +653,15 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity * value.
  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * Custom date formats follow the formats at + * + * Datetime Patterns. * This applies to date type.
  • - *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.time.format.DateTimeFormatter`. This applies to timestamp type.
  • + * + * Datetime Patterns. + * This applies to timestamp type. *
  • `maxColumns` (default `20480`): defines a hard limit of how many columns * a record can have.
  • *
  • `maxCharsPerColumn` (default `-1`): defines the maximum number of characters allowed 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 22b26ca8164cf..9aef382470bcc 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 @@ -26,18 +26,17 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, InsertIntoStatement, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.connector.catalog.{CatalogPlugin, CatalogV2Implicits, CatalogV2Util, Identifier, SupportsCatalogOptions, SupportsWrite, Table, TableCatalog, TableProvider, V1Table} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateTableAsSelectStatement, InsertIntoStatement, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelectStatement} +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, CatalogV2Implicits, CatalogV2Util, Identifier, SupportsCatalogOptions, Table, TableCatalog, TableProvider, V1Table} import org.apache.spark.sql.connector.catalog.TableCapability._ -import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, LiteralValue, Transform} +import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, DataSourceUtils, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap /** @@ -107,8 +106,19 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to format timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to format timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 1.4.0 @@ -144,8 +154,19 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to format timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to format timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 1.4.0 @@ -160,8 +181,19 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to format timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to format timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 1.4.0 @@ -198,7 +230,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Buckets the output by the given columns. If specified, the output is laid out on the file - * system similar to Hive's bucketing scheme. + * system similar to Hive's bucketing scheme, but with a different bucket hash function + * and is not compatible with Hive's bucketing. * * This is applicable for all file-based data sources (e.g. Parquet, JSON) starting with Spark * 2.1.0. @@ -540,12 +573,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val canUseV2 = lookupV2Provider().isDefined session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { - case NonSessionCatalogAndIdentifier(catalog, ident) => - saveAsTable(catalog.asTableCatalog, ident) + case nameParts @ NonSessionCatalogAndIdentifier(catalog, ident) => + saveAsTable(catalog.asTableCatalog, ident, nameParts) - case SessionCatalogAndIdentifier(catalog, ident) + case nameParts @ SessionCatalogAndIdentifier(catalog, ident) if canUseV2 && ident.namespace().length <= 1 => - saveAsTable(catalog.asTableCatalog, ident) + saveAsTable(catalog.asTableCatalog, ident, nameParts) case AsTableIdentifier(tableIdentifier) => saveAsTable(tableIdentifier) @@ -557,16 +590,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } - private def saveAsTable(catalog: TableCatalog, ident: Identifier): Unit = { + private def saveAsTable( + catalog: TableCatalog, ident: Identifier, nameParts: Seq[String]): Unit = { val tableOpt = try Option(catalog.loadTable(ident)) catch { case _: NoSuchTableException => None } - def getLocationIfExists: Option[(String, String)] = { - val opts = CaseInsensitiveMap(extraOptions.toMap) - opts.get("path").map(TableCatalog.PROP_LOCATION -> _) - } - val command = (mode, tableOpt) match { case (_, Some(_: V1Table)) => return saveAsTable(TableIdentifier(ident.name(), ident.namespace().headOption)) @@ -577,12 +606,16 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { AppendData.byName(v2Relation, df.logicalPlan, extraOptions.toMap) case (SaveMode.Overwrite, _) => - ReplaceTableAsSelect( - catalog, - ident, - partitioningAsV2, + ReplaceTableAsSelectStatement( + nameParts, df.queryExecution.analyzed, - Map(TableCatalog.PROP_PROVIDER -> source) ++ getLocationIfExists, + partitioningAsV2, + None, + Map.empty, + Some(source), + Map.empty, + extraOptions.get("path"), + extraOptions.get(TableCatalog.PROP_COMMENT), extraOptions.toMap, orCreate = true) // Create the table if it doesn't exist @@ -590,14 +623,18 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { // We have a potential race condition here in AppendMode, if the table suddenly gets // created between our existence check and physical execution, but this can't be helped // in any case. - CreateTableAsSelect( - catalog, - ident, - partitioningAsV2, + CreateTableAsSelectStatement( + nameParts, df.queryExecution.analyzed, - Map(TableCatalog.PROP_PROVIDER -> source) ++ getLocationIfExists, + partitioningAsV2, + None, + Map.empty, + Some(source), + Map.empty, + extraOptions.get("path"), + extraOptions.get(TableCatalog.PROP_COMMENT), extraOptions.toMap, - ignoreIfExists = other == SaveMode.Ignore) + ifNotExists = other == SaveMode.Ignore) } runCommand(df.sparkSession, "saveAsTable") { @@ -749,11 +786,15 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * Custom date formats follow the formats at + * + * Datetime Patterns. * This applies to date type.
  • - *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.time.format.DateTimeFormatter`. This applies to timestamp type.
  • + * + * Datetime Patterns. + * This applies to timestamp type. *
  • `encoding` (by default it is not set): specifies encoding (charset) of saved json * files. If it is not set, the UTF-8 charset will be used.
  • *
  • `lineSep` (default `\n`): defines the line separator that should be used for writing.
  • @@ -870,11 +911,15 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`). *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * Custom date formats follow the formats at + * + * Datetime Patterns. * This applies to date type.
  • - *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.time.format.DateTimeFormatter`. This applies to timestamp type.
  • + * + * Datetime Patterns. + * This applies to timestamp type. *
  • `ignoreLeadingWhiteSpace` (default `true`): a flag indicating whether or not leading * whitespaces from values being written should be skipped.
  • *
  • `ignoreTrailingWhiteSpace` (default `true`): a flag indicating defines whether or not diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index 45a9b28154028..15ff7fa0530cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -23,8 +23,7 @@ import scala.collection.mutable import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.{Attribute, Bucket, Days, Hours, Literal, Months, Years} -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} -import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelectStatement, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelectStatement} import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference, Transform} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation @@ -47,8 +46,6 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private val sparkSession = ds.sparkSession - private val catalogManager = sparkSession.sessionState.analyzer.catalogManager - private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) private val (catalog, identifier) = { @@ -120,19 +117,19 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) } override def create(): Unit = { - // create and replace could alternatively create ParsedPlan statements, like - // `CreateTableFromDataFrameStatement(UnresolvedRelation(tableName), ...)`, to keep the catalog - // resolution logic in the analyzer. runCommand("create") { - CreateTableAsSelect( - catalog, - identifier, - partitioning.getOrElse(Seq.empty), + CreateTableAsSelectStatement( + tableName, logicalPlan, - properties = provider.map(p => properties + (TableCatalog.PROP_PROVIDER -> p)) - .getOrElse(properties).toMap, - writeOptions = options.toMap, - ignoreIfExists = false) + partitioning.getOrElse(Seq.empty), + None, + properties.toMap, + provider, + Map.empty, + None, + None, + options.toMap, + ifNotExists = false) } } @@ -231,13 +228,17 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private def internalReplace(orCreate: Boolean): Unit = { runCommand("replace") { - ReplaceTableAsSelect( - catalog, - identifier, - partitioning.getOrElse(Seq.empty), + ReplaceTableAsSelectStatement( + tableName, logicalPlan, - properties = provider.map(p => properties + ("provider" -> p)).getOrElse(properties).toMap, - writeOptions = options.toMap, + partitioning.getOrElse(Seq.empty), + None, + properties.toMap, + provider, + Map.empty, + None, + None, + options.toMap, orCreate = orCreate) } } 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 e3c63881b07ac..c897170c91faa 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 @@ -22,6 +22,7 @@ import java.io.{ByteArrayOutputStream, CharArrayWriter, DataOutputStream} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions +import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import org.apache.commons.lang3.StringUtils @@ -34,6 +35,7 @@ import org.apache.spark.api.python.{PythonRDD, SerDeUtil} import org.apache.spark.api.r.RRDD import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.HiveTableRelation @@ -2265,6 +2267,90 @@ class Dataset[T] private[sql]( randomSplit(weights.toArray, seed) } + /** + * (Scala-specific) Returns a new Dataset where each row has been expanded to zero or more + * rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of + * the input row are implicitly joined with each row that is output by the function. + * + * Given that this is deprecated, as an alternative, you can explode columns either using + * `functions.explode()` or `flatMap()`. The following example uses these alternatives to count + * the number of books that contain a given word: + * + * {{{ + * case class Book(title: String, words: String) + * val ds: Dataset[Book] + * + * val allWords = ds.select('title, explode(split('words, " ")).as("word")) + * + * val bookCountPerWord = allWords.groupBy("word").agg(countDistinct("title")) + * }}} + * + * Using `flatMap()` this can similarly be exploded as: + * + * {{{ + * ds.flatMap(_.words.split(" ")) + * }}} + * + * @group untypedrel + * @since 2.0.0 + */ + @deprecated("use flatMap() or select() with functions.explode() instead", "2.0.0") + def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { + val elementSchema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] + + val convert = CatalystTypeConverters.createToCatalystConverter(elementSchema) + + val rowFunction = + f.andThen(_.map(convert(_).asInstanceOf[InternalRow])) + val generator = UserDefinedGenerator(elementSchema, rowFunction, input.map(_.expr)) + + withPlan { + Generate(generator, unrequiredChildIndex = Nil, outer = false, + qualifier = None, generatorOutput = Nil, logicalPlan) + } + } + + /** + * (Scala-specific) Returns a new Dataset where a single column has been expanded to zero + * or more rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. All + * columns of the input row are implicitly joined with each value that is output by the function. + * + * Given that this is deprecated, as an alternative, you can explode columns either using + * `functions.explode()`: + * + * {{{ + * ds.select(explode(split('words, " ")).as("word")) + * }}} + * + * or `flatMap()`: + * + * {{{ + * ds.flatMap(_.words.split(" ")) + * }}} + * + * @group untypedrel + * @since 2.0.0 + */ + @deprecated("use flatMap() or select() with functions.explode() instead", "2.0.0") + def explode[A, B : TypeTag](inputColumn: String, outputColumn: String)(f: A => TraversableOnce[B]) + : DataFrame = { + val dataType = ScalaReflection.schemaFor[B].dataType + val attributes = AttributeReference(outputColumn, dataType)() :: Nil + // TODO handle the metadata? + val elementSchema = attributes.toStructType + + def rowFunction(row: Row): TraversableOnce[InternalRow] = { + val convert = CatalystTypeConverters.createToCatalystConverter(dataType) + f(row(0).asInstanceOf[A]).map(o => InternalRow(convert(o))) + } + val generator = UserDefinedGenerator(elementSchema, rowFunction, apply(inputColumn).expr :: Nil) + + withPlan { + Generate(generator, unrequiredChildIndex = Nil, outer = false, + qualifier = None, generatorOutput = Nil, logicalPlan) + } + } + /** * Returns a new Dataset by adding a column or replacing the existing column that has * the same name. @@ -2455,7 +2541,7 @@ class Dataset[T] private[sql]( def dropDuplicates(colNames: Seq[String]): Dataset[T] = withTypedPlan { val resolver = sparkSession.sessionState.analyzer.resolver val allColumns = queryExecution.analyzed.output - val groupCols = colNames.toSet.toSeq.flatMap { (colName: String) => + val groupCols = colNames.distinct.flatMap { (colName: String) => // It is possibly there are more than one columns with the same name, // so we call filter instead of find. val cols = allColumns.filter(col => resolver(col.name, colName)) @@ -3129,6 +3215,18 @@ class Dataset[T] private[sql]( */ def javaRDD: JavaRDD[T] = toJavaRDD + /** + * Registers this Dataset as a temporary table using the given name. The lifetime of this + * temporary table is tied to the [[SparkSession]] that was used to create this Dataset. + * + * @group basic + * @since 1.6.0 + */ + @deprecated("Use createOrReplaceTempView(viewName) instead.", "2.0.0") + def registerTempTable(tableName: String): Unit = { + createOrReplaceTempView(tableName) + } + /** * Creates a local temporary view using the given name. The lifetime of this * temporary view is tied to the [[SparkSession]] that was used to create this Dataset. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 2054874e5e07b..68ce82d5badda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -24,7 +24,7 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} -import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.rdd.RDD @@ -64,6 +64,15 @@ class SQLContext private[sql](val sparkSession: SparkSession) // Note: Since Spark 2.0 this class has become a wrapper of SparkSession, where the // real functionality resides. This class remains mainly for backward compatibility. + + @deprecated("Use SparkSession.builder instead", "2.0.0") + def this(sc: SparkContext) = { + this(SparkSession.builder().sparkContext(sc).getOrCreate()) + } + + @deprecated("Use SparkSession.builder instead", "2.0.0") + def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) + // TODO: move this logic into SparkSession private[sql] def sessionState: SessionState = sparkSession.sessionState @@ -479,6 +488,97 @@ class SQLContext private[sql](val sparkSession: SparkSession) def readStream: DataStreamReader = sparkSession.readStream + /** + * Creates an external 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 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable(tableName: String, path: String): DataFrame = { + sparkSession.catalog.createTable(tableName, path) + } + + /** + * Creates an external table from the given path based on a data source + * and returns the corresponding DataFrame. + * + * @group ddl_ops + * @since 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + path: String, + source: String): DataFrame = { + sparkSession.catalog.createTable(tableName, path, source) + } + + /** + * 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 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + options: java.util.Map[String, String]): DataFrame = { + sparkSession.catalog.createTable(tableName, source, options) + } + + /** + * (Scala-specific) + * 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 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + options: Map[String, String]): DataFrame = { + sparkSession.catalog.createTable(tableName, source, options) + } + + /** + * 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 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: java.util.Map[String, String]): DataFrame = { + sparkSession.catalog.createTable(tableName, source, schema, options) + } + + /** + * (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. + * + * @group ddl_ops + * @since 1.3.0 + */ + @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: Map[String, String]): DataFrame = { + sparkSession.catalog.createTable(tableName, source, schema, options) + } + /** * Registers the given `DataFrame` as a temporary table in the catalog. Temporary tables exist * only during the lifetime of this instance of SQLContext. @@ -611,6 +711,289 @@ class SQLContext private[sql](val sparkSession: SparkSession) sessionState.catalog.listTables(databaseName).map(_.table).toArray } + //////////////////////////////////////////////////////////////////////////// + //////////////////////////////////////////////////////////////////////////// + // Deprecated methods + //////////////////////////////////////////////////////////////////////////// + //////////////////////////////////////////////////////////////////////////// + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rowRDD: RDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema) + } + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema) + } + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rdd: RDD[_], beanClass: Class[_]): DataFrame = { + createDataFrame(rdd, beanClass) + } + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = { + createDataFrame(rdd, beanClass) + } + + /** + * Loads a Parquet file, returning the result as a `DataFrame`. This function returns an empty + * `DataFrame` if no paths are passed in. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().parquet()`. + */ + @deprecated("Use read.parquet() instead.", "1.4.0") + @scala.annotation.varargs + def parquetFile(paths: String*): DataFrame = { + if (paths.isEmpty) { + emptyDataFrame + } else { + read.parquet(paths : _*) + } + } + + /** + * Loads a JSON file (one object per line), returning the result as a `DataFrame`. + * It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String): DataFrame = { + read.json(path) + } + + /** + * Loads a JSON file (one object per line) and applies the given schema, + * returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String, schema: StructType): DataFrame = { + read.schema(schema).json(path) + } + + /** + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String, samplingRatio: Double): DataFrame = { + read.option("samplingRatio", samplingRatio.toString).json(path) + } + + /** + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a + * `DataFrame`. + * It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String]): DataFrame = read.json(json) + + /** + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a + * `DataFrame`. + * It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String]): DataFrame = read.json(json) + + /** + * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, + * returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String], schema: StructType): DataFrame = { + read.schema(schema).json(json) + } + + /** + * Loads an JavaRDD[String] storing JSON objects (one object per record) and applies the given + * schema, returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String], schema: StructType): DataFrame = { + read.schema(schema).json(json) + } + + /** + * Loads an RDD[String] storing JSON objects (one object per record) inferring the + * schema, returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String], samplingRatio: Double): DataFrame = { + read.option("samplingRatio", samplingRatio.toString).json(json) + } + + /** + * Loads a JavaRDD[String] storing JSON objects (one object per record) inferring the + * schema, returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String], samplingRatio: Double): DataFrame = { + read.option("samplingRatio", samplingRatio.toString).json(json) + } + + /** + * Returns the dataset stored at path as a DataFrame, + * using the default data source configured by spark.sql.sources.default. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().load(path)`. + */ + @deprecated("Use read.load(path) instead.", "1.4.0") + def load(path: String): DataFrame = { + read.load(path) + } + + /** + * Returns the dataset stored at path as a DataFrame, using the given data source. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().format(source).load(path)`. + */ + @deprecated("Use read.format(source).load(path) instead.", "1.4.0") + def load(path: String, source: String): DataFrame = { + read.format(source).load(path) + } + + /** + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().format(source).options(options).load()`. + */ + @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") + def load(source: String, options: java.util.Map[String, String]): DataFrame = { + read.options(options).format(source).load() + } + + /** + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().format(source).options(options).load()`. + */ + @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") + def load(source: String, options: Map[String, String]): DataFrame = { + read.options(options).format(source).load() + } + + /** + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by + * `read().format(source).schema(schema).options(options).load()`. + */ + @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") + def load( + source: String, + schema: StructType, + options: java.util.Map[String, String]): DataFrame = { + read.format(source).schema(schema).options(options).load() + } + + /** + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by + * `read().format(source).schema(schema).options(options).load()`. + */ + @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") + def load(source: String, schema: StructType, options: Map[String, String]): DataFrame = { + read.format(source).schema(schema).options(options).load() + } + + /** + * Construct a `DataFrame` representing the database table accessible via JDBC URL + * url named table. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc(url: String, table: String): DataFrame = { + read.jdbc(url, table, new Properties) + } + + /** + * Construct a `DataFrame` representing the database table accessible via JDBC URL + * url named table. Partitions of the table will be retrieved in parallel based on the parameters + * passed to this function. + * + * @param columnName the name of a column of integral type that will be used for partitioning. + * @param lowerBound the minimum value of `columnName` used to decide partition stride + * @param upperBound the maximum value of `columnName` used to decide partition stride + * @param numPartitions the number of partitions. the range `minValue`-`maxValue` will be split + * evenly into this many partitions + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc( + url: String, + table: String, + columnName: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int): DataFrame = { + read.jdbc(url, table, columnName, lowerBound, upperBound, numPartitions, new Properties) + } + + /** + * Construct a `DataFrame` representing the database table accessible via JDBC URL + * url named table. The theParts parameter gives a list expressions + * suitable for inclusion in WHERE clauses; each one defines one partition + * of the `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc(url: String, table: String, theParts: Array[String]): DataFrame = { + read.jdbc(url, table, theParts, new Properties) + } } /** @@ -623,6 +1006,45 @@ class SQLContext private[sql](val sparkSession: SparkSession) */ object SQLContext { + /** + * Get the singleton SQLContext if it exists or create a new one using the given SparkContext. + * + * This function can be used to create a singleton SQLContext object that can be shared across + * the JVM. + * + * If there is an active SQLContext for current thread, it will be returned instead of the global + * one. + * + * @since 1.5.0 + */ + @deprecated("Use SparkSession.builder instead", "2.0.0") + def getOrCreate(sparkContext: SparkContext): SQLContext = { + SparkSession.builder().sparkContext(sparkContext).getOrCreate().sqlContext + } + + /** + * Changes the SQLContext that will be returned in this thread and its children when + * SQLContext.getOrCreate() is called. This can be used to ensure that a given thread receives + * a SQLContext with an isolated session, instead of the global (first created) context. + * + * @since 1.6.0 + */ + @deprecated("Use SparkSession.setActiveSession instead", "2.0.0") + def setActive(sqlContext: SQLContext): Unit = { + SparkSession.setActiveSession(sqlContext.sparkSession) + } + + /** + * Clears the active SQLContext for current thread. Subsequent calls to getOrCreate will + * return the first created context instead of a thread-local override. + * + * @since 1.6.0 + */ + @deprecated("Use SparkSession.clearActiveSession instead", "2.0.0") + def clearActive(): Unit = { + SparkSession.clearActiveSession() + } + /** * Converts an iterator of Java Beans to InternalRow using the provided * bean info & schema. This is not related to the singleton, but is a static 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 0f08e10c00d22..ced4af46c3f30 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 @@ -125,7 +125,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends (0 to 22).foreach { x => val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) - val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"Try(ScalaReflection.schemaFor[A$i]).toOption :: $s"}) + val inputEncoders = (1 to x).foldRight("Nil")((i, s) => {s"Try(ExpressionEncoder[A$i]()).toOption :: $s"}) println(s""" |/** | * Registers a deterministic Scala closure of $x arguments as user-defined function (UDF). @@ -134,8 +134,8 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends | */ |def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - | val inputSchemas: Seq[Option[ScalaReflection.Schema]] = $inputSchemas - | val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + | val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = $inputEncoders + | val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) | val finalUdf = if (nullable) udf else udf.asNonNullable() | def builder(e: Seq[Expression]) = if (e.length == $x) { | finalUdf.createScalaUDF(e) @@ -163,7 +163,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { | val func = $funcCall | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + | ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $i; Found: " + e.length) @@ -180,8 +180,8 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 0) { finalUdf.createScalaUDF(e) @@ -200,8 +200,8 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 1) { finalUdf.createScalaUDF(e) @@ -220,8 +220,8 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 2) { finalUdf.createScalaUDF(e) @@ -240,8 +240,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 3) { finalUdf.createScalaUDF(e) @@ -260,8 +260,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 4) { finalUdf.createScalaUDF(e) @@ -280,8 +280,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 5) { finalUdf.createScalaUDF(e) @@ -300,8 +300,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 6) { finalUdf.createScalaUDF(e) @@ -320,8 +320,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 7) { finalUdf.createScalaUDF(e) @@ -340,8 +340,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 8) { finalUdf.createScalaUDF(e) @@ -360,8 +360,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 9) { finalUdf.createScalaUDF(e) @@ -380,8 +380,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 10) { finalUdf.createScalaUDF(e) @@ -400,8 +400,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 11) { finalUdf.createScalaUDF(e) @@ -420,8 +420,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 12) { finalUdf.createScalaUDF(e) @@ -440,8 +440,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 13) { finalUdf.createScalaUDF(e) @@ -460,8 +460,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 14) { finalUdf.createScalaUDF(e) @@ -480,8 +480,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 15) { finalUdf.createScalaUDF(e) @@ -500,8 +500,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 16) { finalUdf.createScalaUDF(e) @@ -520,8 +520,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 17) { finalUdf.createScalaUDF(e) @@ -540,8 +540,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 18) { finalUdf.createScalaUDF(e) @@ -560,8 +560,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Try(ExpressionEncoder[A19]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 19) { finalUdf.createScalaUDF(e) @@ -580,8 +580,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Try(ExpressionEncoder[A19]()).toOption :: Try(ExpressionEncoder[A20]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 20) { finalUdf.createScalaUDF(e) @@ -600,8 +600,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Try(ScalaReflection.schemaFor[A21]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Try(ExpressionEncoder[A19]()).toOption :: Try(ExpressionEncoder[A20]()).toOption :: Try(ExpressionEncoder[A21]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 21) { finalUdf.createScalaUDF(e) @@ -620,8 +620,8 @@ 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 ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Try(ScalaReflection.schemaFor[A21]).toOption :: Try(ScalaReflection.schemaFor[A22]).toOption :: Nil - val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Try(ExpressionEncoder[A11]()).toOption :: Try(ExpressionEncoder[A12]()).toOption :: Try(ExpressionEncoder[A13]()).toOption :: Try(ExpressionEncoder[A14]()).toOption :: Try(ExpressionEncoder[A15]()).toOption :: Try(ExpressionEncoder[A16]()).toOption :: Try(ExpressionEncoder[A17]()).toOption :: Try(ExpressionEncoder[A18]()).toOption :: Try(ExpressionEncoder[A19]()).toOption :: Try(ExpressionEncoder[A20]()).toOption :: Try(ExpressionEncoder[A21]()).toOption :: Try(ExpressionEncoder[A22]()).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputEncoders).withName(name) val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 22) { finalUdf.createScalaUDF(e) @@ -731,7 +731,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF0[_], returnType: DataType): Unit = { val func = () => f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -746,7 +746,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -761,7 +761,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -776,7 +776,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -791,7 +791,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -806,7 +806,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -821,7 +821,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -836,7 +836,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -851,7 +851,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -866,7 +866,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -881,7 +881,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -896,7 +896,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -911,7 +911,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -926,7 +926,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -941,7 +941,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -956,7 +956,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -971,7 +971,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -986,7 +986,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -1001,7 +1001,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -1016,7 +1016,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -1031,7 +1031,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -1046,7 +1046,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -1061,7 +1061,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends 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) def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, Nil, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) 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 318cc629e7a34..60738e6d4ef9e 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalog import scala.collection.JavaConverters._ -import org.apache.spark.annotation.Stable +import org.apache.spark.annotation.{Evolving, Experimental, Stable} import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset} import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel @@ -208,6 +208,20 @@ abstract class Catalog { */ def functionExists(dbName: String, functionName: String): Boolean + /** + * 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. + * + * @param tableName is either a qualified or unqualified name that designates a table. + * If no database identifier is provided, it refers to a table in + * the current database. + * @since 2.0.0 + */ + @deprecated("use createTable instead.", "2.2.0") + def createExternalTable(tableName: String, path: String): DataFrame = { + createTable(tableName, path) + } + /** * 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. @@ -219,6 +233,20 @@ abstract class Catalog { */ def createTable(tableName: String, path: String): DataFrame + /** + * Creates a table from the given path based on a data source and returns the corresponding + * DataFrame. + * + * @param tableName is either a qualified or unqualified name that designates a table. + * If no database identifier is provided, it refers to a table in + * the current database. + * @since 2.0.0 + */ + @deprecated("use createTable instead.", "2.2.0") + def createExternalTable(tableName: String, path: String, source: String): DataFrame = { + createTable(tableName, path, source) + } + /** * Creates a table from the given path based on a data source and returns the corresponding * DataFrame. @@ -230,6 +258,23 @@ abstract class Catalog { */ def createTable(tableName: String, path: String, source: String): DataFrame + /** + * Creates a table from the given path based on a data source and a set of options. + * Then, returns the corresponding DataFrame. + * + * @param tableName is either a qualified or unqualified name that designates a table. + * If no database identifier is provided, it refers to a table in + * the current database. + * @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) + } + /** * Creates a table based on the dataset in a data source and a set of options. * Then, returns the corresponding DataFrame. @@ -246,6 +291,24 @@ abstract class Catalog { 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. + * + * @param tableName is either a qualified or unqualified name that designates a table. + * If no database identifier is provided, it refers to a table in + * the current database. + * @since 2.0.0 + */ + @deprecated("use createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + options: Map[String, String]): DataFrame = { + createTable(tableName, source, options) + } + /** * (Scala-specific) * Creates a table based on the dataset in a data source and a set of options. @@ -261,6 +324,24 @@ abstract class Catalog { source: String, options: Map[String, String]): 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. + * + * @param tableName is either a qualified or unqualified name that designates a table. + * If no database identifier is provided, it refers to a table in + * the current database. + * @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) + } + /** * Create a table based on the dataset in a data source, a schema and a set of options. * Then, returns the corresponding DataFrame. @@ -278,6 +359,25 @@ abstract class Catalog { 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. + * + * @param tableName is either a qualified or unqualified name that designates a table. + * If no database identifier is provided, it refers to a table in + * the current database. + * @since 2.0.0 + */ + @deprecated("use createTable instead.", "2.2.0") + def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: Map[String, String]): DataFrame = { + createTable(tableName, source, schema, options) + } + /** * (Scala-specific) * Create a table based on the dataset in a data source, a schema and a set of options. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 1e14dc8906797..f74d768967449 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -50,6 +50,9 @@ class ResolveSessionCatalog( nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => + if (!DDLUtils.isHiveTable(v1Table.v1Table)) { + cols.foreach(c => failCharType(c.dataType)) + } cols.foreach { c => assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand") if (!c.nullable) { @@ -59,6 +62,7 @@ class ResolveSessionCatalog( } AlterTableAddColumnsCommand(tbl.asTableIdentifier, cols.map(convertToStructField)) }.getOrElse { + cols.foreach(c => failCharType(c.dataType)) val changes = cols.map { col => TableChange.addColumn( col.name.toArray, @@ -76,6 +80,7 @@ class ResolveSessionCatalog( case Some(_: V1Table) => throw new AnalysisException("REPLACE COLUMNS is only supported with v2 tables.") case Some(table) => + cols.foreach(c => failCharType(c.dataType)) // REPLACE COLUMNS deletes all the existing columns and adds new columns specified. val deleteChanges = table.schema.fieldNames.map { name => TableChange.deleteColumn(Array(name)) @@ -97,6 +102,10 @@ class ResolveSessionCatalog( nameParts @ SessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => + if (!DDLUtils.isHiveTable(v1Table.v1Table)) { + a.dataType.foreach(failCharType) + } + if (a.column.length > 1) { throw new AnalysisException( "ALTER COLUMN with qualified column is only supported with v2 tables.") @@ -134,6 +143,7 @@ class ResolveSessionCatalog( builder.build()) AlterTableChangeColumnCommand(tbl.asTableIdentifier, colName, newColumn) }.getOrElse { + a.dataType.foreach(failCharType) val colName = a.column.toArray val typeChange = a.dataType.map { newDataType => TableChange.updateColumnType(colName, newDataType) @@ -260,12 +270,16 @@ class ResolveSessionCatalog( SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => val provider = c.provider.getOrElse(conf.defaultDataSourceName) if (!isV2Provider(provider)) { + if (!DDLUtils.isHiveTable(Some(provider))) { + assertNoCharTypeInSchema(c.tableSchema) + } val tableDesc = buildCatalogTable(tbl.asTableIdentifier, c.tableSchema, c.partitioning, c.bucketSpec, c.properties, provider, c.options, c.location, c.comment, c.ifNotExists) val mode = if (c.ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTable(tableDesc, mode, None) } else { + assertNoCharTypeInSchema(c.tableSchema) CreateV2Table( catalog.asTableCatalog, tbl.asIdentifier, @@ -277,7 +291,7 @@ class ResolveSessionCatalog( } case c @ CreateTableAsSelectStatement( - SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _, _) => val provider = c.provider.getOrElse(conf.defaultDataSourceName) if (!isV2Provider(provider)) { val tableDesc = buildCatalogTable(tbl.asTableIdentifier, new StructType, @@ -293,7 +307,7 @@ class ResolveSessionCatalog( c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, convertTableProperties(c.properties, c.options, c.location, c.comment, Some(provider)), - writeOptions = c.options, + writeOptions = c.writeOptions, ignoreIfExists = c.ifNotExists) } @@ -309,6 +323,7 @@ class ResolveSessionCatalog( if (!isV2Provider(provider)) { throw new AnalysisException("REPLACE TABLE is only supported with v2 tables.") } else { + assertNoCharTypeInSchema(c.tableSchema) ReplaceTable( catalog.asTableCatalog, tbl.asIdentifier, @@ -320,7 +335,7 @@ class ResolveSessionCatalog( } case c @ ReplaceTableAsSelectStatement( - SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => + SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _, _) => val provider = c.provider.getOrElse(conf.defaultDataSourceName) if (!isV2Provider(provider)) { throw new AnalysisException("REPLACE TABLE AS SELECT is only supported with v2 tables.") @@ -332,7 +347,7 @@ class ResolveSessionCatalog( c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, convertTableProperties(c.properties, c.options, c.location, c.comment, Some(provider)), - writeOptions = c.options, + writeOptions = c.writeOptions, orCreate = c.orCreate) } @@ -530,6 +545,20 @@ class ResolveSessionCatalog( replace, viewType) + case ShowViews(resolved: ResolvedNamespace, pattern) => + resolved match { + case SessionCatalogAndNamespace(_, ns) => + // Fallback to v1 ShowViewsCommand since there is no view API in v2 catalog + assert(ns.nonEmpty) + if (ns.length != 1) { + throw new AnalysisException(s"The database name is not valid: ${ns.quoted}") + } + ShowViewsCommand(ns.head, pattern) + case _ => + throw new AnalysisException(s"Catalog ${resolved.catalog.name} doesn't support " + + "SHOW VIEWS, only SessionCatalog supports this command.") + } + case ShowTableProperties(r: ResolvedTable, propertyKey) if isSessionCatalog(r.catalog) => ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala index e482bc9941ea9..e1b9c8f430c56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala @@ -87,7 +87,7 @@ object CollectMetricsExec { * Recursively collect all collected metrics from a query tree. */ def collect(plan: SparkPlan): Map[String, Row] = { - val metrics = plan.collectInPlanAndSubqueries { + val metrics = plan.collectWithSubqueries { case collector: CollectMetricsExec => collector.name -> collector.collectedMetrics } metrics.toMap 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 45d7af2114f0e..90b08ea558044 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 @@ -253,74 +253,75 @@ case class FileSourceScanExec( partitionFilters.exists(ExecSubqueryExpression.hasSubquery) } - override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = { - val bucketSpec = if (relation.sparkSession.sessionState.conf.bucketingEnabled) { - relation.bucketSpec + private def toAttribute(colName: String): Option[Attribute] = + output.find(_.name == colName) + + // exposed for testing + lazy val bucketedScan: Boolean = { + if (relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined) { + val spec = relation.bucketSpec.get + val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) + bucketColumns.size == spec.bucketColumnNames.size } else { - None + false } - bucketSpec match { - case Some(spec) => - // For bucketed columns: - // ----------------------- - // `HashPartitioning` would be used only when: - // 1. ALL the bucketing columns are being read from the table - // - // For sorted columns: - // --------------------- - // Sort ordering should be used when ALL these criteria's match: - // 1. `HashPartitioning` is being used - // 2. A prefix (or all) of the sort columns are being read from the table. - // - // Sort ordering would be over the prefix subset of `sort columns` being read - // from the table. - // eg. - // Assume (col0, col2, col3) are the columns read from the table - // If sort columns are (col0, col1), then sort ordering would be considered as (col0) - // If sort columns are (col1, col0), then sort ordering would be empty as per rule #2 - // above - - def toAttribute(colName: String): Option[Attribute] = - output.find(_.name == colName) - - val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) - if (bucketColumns.size == spec.bucketColumnNames.size) { - val partitioning = HashPartitioning(bucketColumns, spec.numBuckets) - val sortColumns = - spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get) - val shouldCalculateSortOrder = - conf.getConf(SQLConf.LEGACY_BUCKETED_TABLE_SCAN_OUTPUT_ORDERING) && - sortColumns.nonEmpty && - !hasPartitionsAvailableAtRunTime - - val sortOrder = if (shouldCalculateSortOrder) { - // In case of bucketing, its possible to have multiple files belonging to the - // same bucket in a given relation. Each of these files are locally sorted - // but those files combined together are not globally sorted. Given that, - // the RDD partition will not be sorted even if the relation has sort columns set - // Current solution is to check if all the buckets have a single file in it - - val files = selectedPartitions.flatMap(partition => partition.files) - val bucketToFilesGrouping = - files.map(_.getPath.getName).groupBy(file => BucketingUtils.getBucketId(file)) - val singleFilePartitions = bucketToFilesGrouping.forall(p => p._2.length <= 1) - - if (singleFilePartitions) { - // TODO Currently Spark does not support writing columns sorting in descending order - // so using Ascending order. This can be fixed in future - sortColumns.map(attribute => SortOrder(attribute, Ascending)) - } else { - Nil - } - } else { - Nil - } - (partitioning, sortOrder) + } + + override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = { + if (bucketedScan) { + // For bucketed columns: + // ----------------------- + // `HashPartitioning` would be used only when: + // 1. ALL the bucketing columns are being read from the table + // + // For sorted columns: + // --------------------- + // Sort ordering should be used when ALL these criteria's match: + // 1. `HashPartitioning` is being used + // 2. A prefix (or all) of the sort columns are being read from the table. + // + // Sort ordering would be over the prefix subset of `sort columns` being read + // from the table. + // eg. + // Assume (col0, col2, col3) are the columns read from the table + // If sort columns are (col0, col1), then sort ordering would be considered as (col0) + // If sort columns are (col1, col0), then sort ordering would be empty as per rule #2 + // above + val spec = relation.bucketSpec.get + val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) + val partitioning = HashPartitioning(bucketColumns, spec.numBuckets) + val sortColumns = + spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get) + val shouldCalculateSortOrder = + conf.getConf(SQLConf.LEGACY_BUCKETED_TABLE_SCAN_OUTPUT_ORDERING) && + sortColumns.nonEmpty && + !hasPartitionsAvailableAtRunTime + + val sortOrder = if (shouldCalculateSortOrder) { + // In case of bucketing, its possible to have multiple files belonging to the + // same bucket in a given relation. Each of these files are locally sorted + // but those files combined together are not globally sorted. Given that, + // the RDD partition will not be sorted even if the relation has sort columns set + // Current solution is to check if all the buckets have a single file in it + + val files = selectedPartitions.flatMap(partition => partition.files) + val bucketToFilesGrouping = + files.map(_.getPath.getName).groupBy(file => BucketingUtils.getBucketId(file)) + val singleFilePartitions = bucketToFilesGrouping.forall(p => p._2.length <= 1) + + if (singleFilePartitions) { + // TODO Currently Spark does not support writing columns sorting in descending order + // so using Ascending order. This can be fixed in future + sortColumns.map(attribute => SortOrder(attribute, Ascending)) } else { - (UnknownPartitioning(0), Nil) + Nil } - case _ => - (UnknownPartitioning(0), Nil) + } else { + Nil + } + (partitioning, sortOrder) + } else { + (UnknownPartitioning(0), Nil) } } @@ -393,11 +394,11 @@ case class FileSourceScanExec( options = relation.options, hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options)) - val readRDD = relation.bucketSpec match { - case Some(bucketing) if relation.sparkSession.sessionState.conf.bucketingEnabled => - createBucketedReadRDD(bucketing, readFile, dynamicallySelectedPartitions, relation) - case _ => - createNonBucketedReadRDD(readFile, dynamicallySelectedPartitions, relation) + val readRDD = if (bucketedScan) { + createBucketedReadRDD(relation.bucketSpec.get, readFile, dynamicallySelectedPartitions, + relation) + } else { + createNonBucketedReadRDD(readFile, dynamicallySelectedPartitions, relation) } sendDriverMetrics() readRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index 5a2f16d8e1526..1a84db1970449 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -23,7 +23,8 @@ import java.time.{Instant, LocalDate} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} -import org.apache.spark.sql.execution.command.{DescribeCommandBase, ExecutedCommandExec, ShowTablesCommand} +import org.apache.spark.sql.execution.command.{DescribeCommandBase, ExecutedCommandExec, ShowTablesCommand, ShowViewsCommand} +import org.apache.spark.sql.execution.datasources.v2.{DescribeTableExec, ShowTablesExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -38,18 +39,21 @@ object HiveResult { */ def hiveResultString(executedPlan: SparkPlan): Seq[String] = executedPlan match { case ExecutedCommandExec(_: DescribeCommandBase) => - // If it is a describe command for a Hive table, we want to have the output format - // be similar with Hive. - executedPlan.executeCollectPublic().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 output database, table name, isTemp. + formatDescribeTableOutput(executedPlan.executeCollectPublic()) + case _: DescribeTableExec => + formatDescribeTableOutput(executedPlan.executeCollectPublic()) + // SHOW TABLES in Hive only output table names while our v1 command outputs + // database, table name, isTemp. case command @ ExecutedCommandExec(s: ShowTablesCommand) if !s.isExtended => command.executeCollect().map(_.getString(1)) + // SHOW TABLES in Hive only output table names while our v2 command outputs + // namespace and table name. + case command : ShowTablesExec => + command.executeCollect().map(_.getString(1)) + // SHOW VIEWS in Hive only outputs view names while our v1 command outputs + // namespace, viewName, and isTemporary. + case command @ ExecutedCommandExec(_: ShowViewsCommand) => + command.executeCollect().map(_.getString(1)) case other => val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq // We need the types so we can output struct field names @@ -59,9 +63,18 @@ object HiveResult { .map(_.mkString("\t")) } - private lazy val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) - private lazy val dateFormatter = DateFormatter(zoneId) - private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) + private def formatDescribeTableOutput(rows: Array[Row]): Seq[String] = { + rows.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") + } + } + + private def zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) + private def dateFormatter = DateFormatter(zoneId) + private def timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) /** Formats a datum (based on the given data type) and returns the string representation. */ def toHiveString(a: (Any, DataType), nested: Boolean = false): String = a match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 5e4f30a5edaf1..c62670b227bcc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -84,6 +84,9 @@ object SQLExecution { redactedStr.substring(0, Math.min(truncateLength, redactedStr.length)) }.getOrElse(callSite.shortForm) + val planDescriptionMode = + ExplainMode.fromString(sparkSession.sessionState.conf.uiExplainMode) + withSQLConfPropagated(sparkSession) { var ex: Option[Throwable] = None val startTime = System.nanoTime() @@ -92,7 +95,7 @@ object SQLExecution { executionId = executionId, description = desc, details = callSite.longForm, - physicalPlanDescription = queryExecution.toString, + physicalPlanDescription = queryExecution.explainString(planDescriptionMode), // `queryExecution.executedPlan` triggers query planning. If it fails, the exception // will be caught and reported in the `SparkListenerSQLExecutionEnd` sparkPlanInfo = SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index 53ab0493f47eb..5936492dd819c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -31,17 +31,23 @@ sealed trait ShufflePartitionSpec // A partition that reads data of one or more reducers, from `startReducerIndex` (inclusive) to // `endReducerIndex` (exclusive). case class CoalescedPartitionSpec( - startReducerIndex: Int, endReducerIndex: Int) extends ShufflePartitionSpec + startReducerIndex: Int, + endReducerIndex: Int) extends ShufflePartitionSpec // A partition that reads partial data of one reducer, from `startMapIndex` (inclusive) to // `endMapIndex` (exclusive). case class PartialReducerPartitionSpec( - reducerIndex: Int, startMapIndex: Int, endMapIndex: Int) extends ShufflePartitionSpec + reducerIndex: Int, + startMapIndex: Int, + endMapIndex: Int, + @transient dataSize: Long) extends ShufflePartitionSpec // A partition that reads partial data of one mapper, from `startReducerIndex` (inclusive) to // `endReducerIndex` (exclusive). case class PartialMapperPartitionSpec( - mapIndex: Int, startReducerIndex: Int, endReducerIndex: Int) extends ShufflePartitionSpec + mapIndex: Int, + startReducerIndex: Int, + endReducerIndex: Int) extends ShufflePartitionSpec /** * The [[Partition]] used by [[ShuffledRowRDD]]. @@ -162,7 +168,7 @@ class ShuffledRowRDD( tracker.getPreferredLocationsForShuffle(dependency, reducerIndex) } - case PartialReducerPartitionSpec(_, startMapIndex, endMapIndex) => + case PartialReducerPartitionSpec(_, startMapIndex, endMapIndex, _) => tracker.getMapLocation(dependency, startMapIndex, endMapIndex) case PartialMapperPartitionSpec(mapIndex, _, _) => @@ -184,7 +190,7 @@ class ShuffledRowRDD( context, sqlMetricsReporter) - case PartialReducerPartitionSpec(reducerIndex, startMapIndex, endMapIndex) => + case PartialReducerPartitionSpec(reducerIndex, startMapIndex, endMapIndex, _) => SparkEnv.get.shuffleManager.getReaderForRange( dependency.shuffleHandle, startMapIndex, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 10fe0f252322f..9f6e4fcdbdd2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import java.util.Locale -import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import scala.collection.mutable import scala.util.control.NonFatal @@ -567,6 +567,26 @@ object WholeStageCodegenExec { def isTooManyFields(conf: SQLConf, dataType: DataType): Boolean = { numOfNestedFields(dataType) > conf.wholeStageMaxNumFields } + + // The whole-stage codegen generates Java code on the driver side and sends it to the Executors + // for compilation and execution. The whole-stage codegen can bring significant performance + // improvements with large dataset in distributed environments. However, in the test environment, + // due to the small amount of data, the time to generate Java code takes up a major part of the + // entire runtime. So we summarize the total code generation time and output it to the execution + // log for easy analysis and view. + private val _codeGenTime = new AtomicLong + + // Increase the total generation time of Java source code in nanoseconds. + // Visible for testing + def increaseCodeGenTime(time: Long): Unit = _codeGenTime.addAndGet(time) + + // Returns the total generation time of Java source code in nanoseconds. + // Visible for testing + def codeGenTime: Long = _codeGenTime.get + + // Reset generation time of Java source code. + // Visible for testing + def resetCodeGenTime: Unit = _codeGenTime.set(0L) } /** @@ -628,6 +648,7 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) * @return the tuple of the codegen context and the actual generated source. */ def doCodeGen(): (CodegenContext, CodeAndComment) = { + val startTime = System.nanoTime() val ctx = new CodegenContext val code = child.asInstanceOf[CodegenSupport].produce(ctx, this) @@ -678,6 +699,9 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) val cleanedSource = CodeFormatter.stripOverlappingComments( new CodeAndComment(CodeFormatter.stripExtraNewLines(source), ctx.getPlaceHolderToComments())) + val duration = System.nanoTime() - startTime + WholeStageCodegenExec.increaseCodeGenTime(duration) + logDebug(s"\n${CodeFormatter.format(cleanedSource)}") (ctx, cleanedSource) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 68da06de2cb7d..217817ec66e6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -96,13 +96,10 @@ case class AdaptiveSparkPlanExec( // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( ReuseAdaptiveSubquery(conf, context.subqueryCache), - // Here the 'OptimizeSkewedJoin' rule should be executed - // before 'CoalesceShufflePartitions', as the skewed partition handled - // in 'OptimizeSkewedJoin' rule, should be omitted in 'CoalesceShufflePartitions'. - OptimizeSkewedJoin(conf), CoalesceShufflePartitions(context.session), - // The rule of 'OptimizeLocalShuffleReader' need to make use of the 'partitionStartIndices' - // in 'CoalesceShufflePartitions' rule. So it must be after 'CoalesceShufflePartitions' rule. + // The following two rules need to make use of 'CustomShuffleReaderExec.partitionSpecs' + // added by `CoalesceShufflePartitions`. So they must be executed after it. + OptimizeSkewedJoin(conf), OptimizeLocalShuffleReader(conf), ApplyColumnarRulesAndInsertTransitions(conf, context.session.sessionState.columnarRules), CollapseCodegenStages(conf) @@ -503,9 +500,10 @@ case class AdaptiveSparkPlanExec( context.session.sparkContext.listenerBus.post(SparkListenerSQLAdaptiveSQLMetricUpdates( executionId.toLong, newMetrics)) } else { + val planDescriptionMode = ExplainMode.fromString(conf.uiExplainMode) context.session.sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( executionId, - SQLExecution.getQueryExecution(executionId).toString, + SQLExecution.getQueryExecution(executionId).explainString(planDescriptionMode), SparkPlanInfo.fromSparkPlan(this))) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala index 61ae6cb14ccd3..cd87230c593f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala @@ -109,7 +109,7 @@ trait AdaptiveSparkPlanHelper { * Returns a sequence containing the result of applying a partial function to all elements in this * plan, also considering all the plans in its (nested) subqueries */ - def collectInPlanAndSubqueries[B](p: SparkPlan)(f: PartialFunction[SparkPlan, B]): Seq[B] = { + def collectWithSubqueries[B](p: SparkPlan)(f: PartialFunction[SparkPlan, B]): Seq[B] = { (p +: subqueriesAll(p)).flatMap(collect(_)(f)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala index d2a7f6ab5426c..6aa34497c9ea6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.adaptive -import org.apache.spark.MapOutputStatistics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan @@ -54,14 +53,10 @@ case class CoalesceShufflePartitions(session: SparkSession) extends Rule[SparkPl if (!shuffleStages.forall(_.shuffle.canChangeNumPartitions)) { plan } else { - val shuffleMetrics = shuffleStages.map { stage => - assert(stage.resultOption.isDefined, "ShuffleQueryStageExec should already be ready") - stage.resultOption.get.asInstanceOf[MapOutputStatistics] - } - - // `ShuffleQueryStageExec` gives null mapOutputStatistics when the input RDD has 0 partitions, + // `ShuffleQueryStageExec#mapStats` returns None when the input RDD has 0 partitions, // we should skip it when calculating the `partitionStartIndices`. - val validMetrics = shuffleMetrics.filter(_ != null) + val validMetrics = shuffleStages.flatMap(_.mapStats) + // We may have different pre-shuffle partition numbers, don't reduce shuffle partition number // in that case. For example when we union fully aggregated data (data is arranged to a single // partition) and a result of a SortMergeJoin (multiple partitions). @@ -74,8 +69,6 @@ case class CoalesceShufflePartitions(session: SparkSession) extends Rule[SparkPl .getOrElse(session.sparkContext.defaultParallelism) val partitionSpecs = ShufflePartitionsUtil.coalescePartitions( validMetrics.toArray, - firstPartitionIndex = 0, - lastPartitionIndex = distinctNumPreShufflePartitions.head, advisoryTargetSize = conf.getConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES), minNumPartitions = minPartitionNum) // This transformation adds new nodes, so we must use `transformUp` here. @@ -85,7 +78,7 @@ case class CoalesceShufflePartitions(session: SparkSession) extends Rule[SparkPl // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same // number of output partitions. case stage: ShuffleQueryStageExec if stageIds.contains(stage.id) => - CustomShuffleReaderExec(stage, partitionSpecs, COALESCED_SHUFFLE_READER_DESCRIPTION) + CustomShuffleReaderExec(stage, partitionSpecs) } } else { plan @@ -93,7 +86,3 @@ case class CoalesceShufflePartitions(session: SparkSession) extends Rule[SparkPl } } } - -object CoalesceShufflePartitions { - val COALESCED_SHUFFLE_READER_DESCRIPTION = "coalesced" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala index ba3f72592908e..08d58e0abdd05 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} /** @@ -31,12 +32,15 @@ import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExcha * @param child It is usually `ShuffleQueryStageExec`, but can be the shuffle exchange * node during canonicalization. * @param partitionSpecs The partition specs that defines the arrangement. - * @param description The string description of this shuffle reader. */ case class CustomShuffleReaderExec private( child: SparkPlan, - partitionSpecs: Seq[ShufflePartitionSpec], - description: String) extends UnaryExecNode { + partitionSpecs: Seq[ShufflePartitionSpec]) extends UnaryExecNode { + // If this reader is to read shuffle files locally, then all partition specs should be + // `PartialMapperPartitionSpec`. + if (partitionSpecs.exists(_.isInstanceOf[PartialMapperPartitionSpec])) { + assert(partitionSpecs.forall(_.isInstanceOf[PartialMapperPartitionSpec])) + } override def output: Seq[Attribute] = child.output override lazy val outputPartitioning: Partitioning = { @@ -62,20 +66,105 @@ case class CustomShuffleReaderExec private( } } - override def stringArgs: Iterator[Any] = Iterator(description) + override def stringArgs: Iterator[Any] = { + val desc = if (isLocalReader) { + "local" + } else if (hasCoalescedPartition && hasSkewedPartition) { + "coalesced and skewed" + } else if (hasCoalescedPartition) { + "coalesced" + } else if (hasSkewedPartition) { + "skewed" + } else { + "" + } + Iterator(desc) + } - private var cachedShuffleRDD: RDD[InternalRow] = null + def hasCoalescedPartition: Boolean = + partitionSpecs.exists(_.isInstanceOf[CoalescedPartitionSpec]) - override protected def doExecute(): RDD[InternalRow] = { - if (cachedShuffleRDD == null) { - cachedShuffleRDD = child match { - case stage: ShuffleQueryStageExec => - new ShuffledRowRDD( - stage.shuffle.shuffleDependency, stage.shuffle.readMetrics, partitionSpecs.toArray) - case _ => - throw new IllegalStateException("operating on canonicalization plan") + def hasSkewedPartition: Boolean = + partitionSpecs.exists(_.isInstanceOf[PartialReducerPartitionSpec]) + + def isLocalReader: Boolean = + partitionSpecs.exists(_.isInstanceOf[PartialMapperPartitionSpec]) + + private def shuffleStage = child match { + case stage: ShuffleQueryStageExec => Some(stage) + case _ => None + } + + private def partitionDataSizeMetrics = { + val maxSize = SQLMetrics.createSizeMetric(sparkContext, "maximum partition data size") + val minSize = SQLMetrics.createSizeMetric(sparkContext, "minimum partition data size") + val avgSize = SQLMetrics.createSizeMetric(sparkContext, "average partition data size") + val mapStatsOpt = shuffleStage.get.mapStats + val sizes = mapStatsOpt.map { mapStats => + val mapSizes = mapStats.bytesByPartitionId + partitionSpecs.map { + case CoalescedPartitionSpec(startReducerIndex, endReducerIndex) => + startReducerIndex.until(endReducerIndex).map(mapSizes).sum + case p: PartialReducerPartitionSpec => p.dataSize + case p => throw new IllegalStateException("unexpected " + p) + } + }.getOrElse(Seq(0L)) + + maxSize.set(sizes.max) + minSize.set(sizes.min) + avgSize.set(sizes.sum / sizes.length) + Map( + "maxPartitionDataSize" -> maxSize, + "minPartitionDataSize" -> minSize, + "avgPartitionDataSize" -> avgSize) + } + + private def skewedPartitionMetrics = { + val metrics = SQLMetrics.createMetric(sparkContext, "number of skewed partitions") + val numSkewedPartitions = partitionSpecs.collect { + case p: PartialReducerPartitionSpec => p.reducerIndex + }.distinct.length + metrics.set(numSkewedPartitions) + Map("numSkewedPartitions" -> metrics) + } + + @transient override lazy val metrics: Map[String, SQLMetric] = { + if (shuffleStage.isDefined) { + val numPartitions = SQLMetrics.createMetric(sparkContext, "number of partitions") + numPartitions.set(partitionSpecs.length) + Map("numPartitions" -> numPartitions) ++ { + if (isLocalReader) { + // We split the mapper partition evenly when creating local shuffle reader, so no + // data size info is available. + Map.empty + } else { + partitionDataSizeMetrics + } + } ++ { + if (hasSkewedPartition) { + skewedPartitionMetrics + } else { + Map.empty + } } + } else { + // It's a canonicalized plan, no need to report metrics. + Map.empty } + } + + private lazy val cachedShuffleRDD: RDD[InternalRow] = { + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) + shuffleStage.map { stage => + new ShuffledRowRDD( + stage.shuffle.shuffleDependency, stage.shuffle.readMetrics, partitionSpecs.toArray) + }.getOrElse { + throw new IllegalStateException("operating on canonicalized plan") + } + } + + override protected def doExecute(): RDD[InternalRow] = { cachedShuffleRDD } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala index e5642991c59a3..0f2868e41cc39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.adaptive -import org.apache.spark.MapOutputStatistics import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, Join, LogicalPlan, NO_BROADCAST_HASH} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf @@ -30,10 +29,10 @@ case class DemoteBroadcastHashJoin(conf: SQLConf) extends Rule[LogicalPlan] { private def shouldDemote(plan: LogicalPlan): Boolean = plan match { case LogicalQueryStage(_, stage: ShuffleQueryStageExec) if stage.resultOption.isDefined - && stage.resultOption.get != null => - val mapOutputStatistics = stage.resultOption.get.asInstanceOf[MapOutputStatistics] - val partitionCnt = mapOutputStatistics.bytesByPartitionId.length - val nonZeroCnt = mapOutputStatistics.bytesByPartitionId.count(_ > 0) + && stage.mapStats.isDefined => + val mapStats = stage.mapStats.get + val partitionCnt = mapStats.bytesByPartitionId.length + val nonZeroCnt = mapStats.bytesByPartitionId.count(_ > 0) partitionCnt > 0 && nonZeroCnt > 0 && (nonZeroCnt * 1.0 / partitionCnt) < conf.nonEmptyPartitionRatioForBroadcastJoin case _ => false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index 621c063e5a7d8..ea586f0bb2561 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -25,7 +25,8 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.command.ExecutedCommandExec +import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedCommandExec} +import org.apache.spark.sql.execution.datasources.v2.V2CommandExec import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.internal.SQLConf @@ -45,6 +46,8 @@ case class InsertAdaptiveSparkPlan( private def applyInternal(plan: SparkPlan, isSubquery: Boolean): SparkPlan = plan match { case _ if !conf.adaptiveExecutionEnabled => plan case _: ExecutedCommandExec => plan + case c: DataWritingCommandExec => c.copy(child = apply(c.child)) + case c: V2CommandExec => c.withNewChildren(c.children.map(apply)) case _ if shouldApplyAQE(plan, isSubquery) => if (supportAdaptive(plan)) { try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index fb6b40c710929..a5b3cac4dfcd0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -65,11 +65,10 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { private def createLocalReader(plan: SparkPlan): CustomShuffleReaderExec = { plan match { - case c @ CustomShuffleReaderExec(s: ShuffleQueryStageExec, _, _) => - CustomShuffleReaderExec( - s, getPartitionSpecs(s, Some(c.partitionSpecs.length)), LOCAL_SHUFFLE_READER_DESCRIPTION) + case c @ CustomShuffleReaderExec(s: ShuffleQueryStageExec, _) => + CustomShuffleReaderExec(s, getPartitionSpecs(s, Some(c.partitionSpecs.length))) case s: ShuffleQueryStageExec => - CustomShuffleReaderExec(s, getPartitionSpecs(s, None), LOCAL_SHUFFLE_READER_DESCRIPTION) + CustomShuffleReaderExec(s, getPartitionSpecs(s, None)) } } @@ -123,8 +122,6 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { object OptimizeLocalShuffleReader { - val LOCAL_SHUFFLE_READER_DESCRIPTION: String = "local" - object BroadcastJoinWithShuffleLeft { def unapply(plan: SparkPlan): Option[(SparkPlan, BuildSide)] = plan match { case join: BroadcastHashJoinExec if canUseLocalShuffleReader(join.left) => @@ -142,8 +139,10 @@ object OptimizeLocalShuffleReader { } def canUseLocalShuffleReader(plan: SparkPlan): Boolean = plan match { - case s: ShuffleQueryStageExec => s.shuffle.canChangeNumPartitions - case CustomShuffleReaderExec(s: ShuffleQueryStageExec, _, _) => s.shuffle.canChangeNumPartitions + case s: ShuffleQueryStageExec => + s.shuffle.canChangeNumPartitions + case CustomShuffleReaderExec(s: ShuffleQueryStageExec, _) => + s.shuffle.canChangeNumPartitions case _ => false } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala index a75a3f37f6672..58e07fabc3973 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import org.apache.commons.io.FileUtils -import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, SparkContext, SparkEnv} +import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, SparkEnv} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ @@ -67,7 +67,7 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { */ private def isSkewed(size: Long, medianSize: Long): Boolean = { size > medianSize * conf.getConf(SQLConf.SKEW_JOIN_SKEWED_PARTITION_FACTOR) && - size > conf.getConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES) + size > conf.getConf(SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD) } private def medianSize(stats: MapOutputStatistics): Long = { @@ -83,14 +83,14 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { /** * The goal of skew join optimization is to make the data distribution more even. The target size * to split skewed partitions is the average size of non-skewed partition, or the - * target post-shuffle partition size if avg size is smaller than it. + * advisory partition size if avg size is smaller than it. */ - private def targetSize(stats: MapOutputStatistics, medianSize: Long): Long = { - val targetPostShuffleSize = conf.getConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES) - val nonSkewSizes = stats.bytesByPartitionId.filterNot(isSkewed(_, medianSize)) + private def targetSize(sizes: Seq[Long], medianSize: Long): Long = { + val advisorySize = conf.getConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES) + val nonSkewSizes = sizes.filterNot(isSkewed(_, medianSize)) // It's impossible that all the partitions are skewed, as we use median size to define skew. assert(nonSkewSizes.nonEmpty) - math.max(targetPostShuffleSize, nonSkewSizes.sum / nonSkewSizes.length) + math.max(advisorySize, nonSkewSizes.sum / nonSkewSizes.length) } /** @@ -102,21 +102,30 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { } /** - * Split the skewed partition based on the map size and the max split number. + * Splits the skewed partition based on the map size and the target partition size + * after split, and create a list of `PartialMapperPartitionSpec`. Returns None if can't split. */ - private def getMapStartIndices( - stage: ShuffleQueryStageExec, - partitionId: Int, - targetSize: Long): Seq[Int] = { - val shuffleId = stage.shuffle.shuffleDependency.shuffleHandle.shuffleId - val mapPartitionSizes = getMapSizesForReduceId(shuffleId, partitionId) - ShufflePartitionsUtil.splitSizeListByTargetSize(mapPartitionSizes, targetSize) - } - - private def getStatistics(stage: ShuffleQueryStageExec): MapOutputStatistics = { - assert(stage.resultOption.isDefined, "ShuffleQueryStageExec should" + - " already be ready when executing OptimizeSkewedPartitions rule") - stage.resultOption.get.asInstanceOf[MapOutputStatistics] + private def createSkewPartitionSpecs( + shuffleId: Int, + reducerId: Int, + targetSize: Long): Option[Seq[PartialReducerPartitionSpec]] = { + val mapPartitionSizes = getMapSizesForReduceId(shuffleId, reducerId) + val mapStartIndices = ShufflePartitionsUtil.splitSizeListByTargetSize( + mapPartitionSizes, targetSize) + if (mapStartIndices.length > 1) { + Some(mapStartIndices.indices.map { i => + val startMapIndex = mapStartIndices(i) + val endMapIndex = if (i == mapStartIndices.length - 1) { + mapPartitionSizes.length + } else { + mapStartIndices(i + 1) + } + val dataSize = startMapIndex.until(endMapIndex).map(mapPartitionSizes(_)).sum + PartialReducerPartitionSpec(reducerId, startMapIndex, endMapIndex, dataSize) + }) + } else { + None + } } private def canSplitLeftSide(joinType: JoinType) = { @@ -128,12 +137,9 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { joinType == Inner || joinType == Cross || joinType == RightOuter } - private def getNumMappers(stage: ShuffleQueryStageExec): Int = { - stage.shuffle.shuffleDependency.rdd.partitions.length - } - - private def getSizeInfo(medianSize: Long, maxSize: Long): String = { - s"median size: $medianSize, max size: ${maxSize}" + private def getSizeInfo(medianSize: Long, sizes: Seq[Long]): String = { + s"median size: $medianSize, max size: ${sizes.max}, min size: ${sizes.min}, avg size: " + + sizes.sum / sizes.length } /* @@ -150,101 +156,91 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { */ def optimizeSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { case smj @ SortMergeJoinExec(_, _, joinType, _, - s1 @ SortExec(_, _, left: ShuffleQueryStageExec, _), - s2 @ SortExec(_, _, right: ShuffleQueryStageExec, _), _) + s1 @ SortExec(_, _, ShuffleStage(left: ShuffleStageInfo), _), + s2 @ SortExec(_, _, ShuffleStage(right: ShuffleStageInfo), _), _) if supportedJoinTypes.contains(joinType) => - val leftStats = getStatistics(left) - val rightStats = getStatistics(right) - val numPartitions = leftStats.bytesByPartitionId.length - - val leftMedSize = medianSize(leftStats) - val rightMedSize = medianSize(rightStats) + assert(left.partitionsWithSizes.length == right.partitionsWithSizes.length) + val numPartitions = left.partitionsWithSizes.length + // We use the median size of the original shuffle partitions to detect skewed partitions. + val leftMedSize = medianSize(left.mapStats) + val rightMedSize = medianSize(right.mapStats) logDebug( s""" - |Try to optimize skewed join. - |Left side partition size: - |${getSizeInfo(leftMedSize, leftStats.bytesByPartitionId.max)} - |Right side partition size: - |${getSizeInfo(rightMedSize, rightStats.bytesByPartitionId.max)} + |Optimizing skewed join. + |Left side partitions size info: + |${getSizeInfo(leftMedSize, left.mapStats.bytesByPartitionId)} + |Right side partitions size info: + |${getSizeInfo(rightMedSize, right.mapStats.bytesByPartitionId)} """.stripMargin) val canSplitLeft = canSplitLeftSide(joinType) val canSplitRight = canSplitRightSide(joinType) - val leftTargetSize = targetSize(leftStats, leftMedSize) - val rightTargetSize = targetSize(rightStats, rightMedSize) + // We use the actual partition sizes (may be coalesced) to calculate target size, so that + // the final data distribution is even (coalesced partitions + split partitions). + val leftActualSizes = left.partitionsWithSizes.map(_._2) + val rightActualSizes = right.partitionsWithSizes.map(_._2) + val leftTargetSize = targetSize(leftActualSizes, leftMedSize) + val rightTargetSize = targetSize(rightActualSizes, rightMedSize) val leftSidePartitions = mutable.ArrayBuffer.empty[ShufflePartitionSpec] val rightSidePartitions = mutable.ArrayBuffer.empty[ShufflePartitionSpec] - // This is used to delay the creation of non-skew partitions so that we can potentially - // coalesce them like `CoalesceShufflePartitions` does. - val nonSkewPartitionIndices = mutable.ArrayBuffer.empty[Int] - val leftSkewDesc = new SkewDesc - val rightSkewDesc = new SkewDesc + var numSkewedLeft = 0 + var numSkewedRight = 0 for (partitionIndex <- 0 until numPartitions) { - val leftSize = leftStats.bytesByPartitionId(partitionIndex) - val isLeftSkew = isSkewed(leftSize, leftMedSize) && canSplitLeft - val rightSize = rightStats.bytesByPartitionId(partitionIndex) - val isRightSkew = isSkewed(rightSize, rightMedSize) && canSplitRight - if (isLeftSkew || isRightSkew) { - if (nonSkewPartitionIndices.nonEmpty) { - // As soon as we see a skew, we'll "flush" out unhandled non-skew partitions. - createNonSkewPartitions(leftStats, rightStats, nonSkewPartitionIndices).foreach { p => - leftSidePartitions += p - rightSidePartitions += p - } - nonSkewPartitionIndices.clear() - } - - val leftParts = if (isLeftSkew) { - val mapStartIndices = getMapStartIndices(left, partitionIndex, leftTargetSize) - if (mapStartIndices.length > 1) { - leftSkewDesc.addPartitionSize(leftSize) - createSkewPartitions(partitionIndex, mapStartIndices, getNumMappers(left)) - } else { - Seq(CoalescedPartitionSpec(partitionIndex, partitionIndex + 1)) - } - } else { - Seq(CoalescedPartitionSpec(partitionIndex, partitionIndex + 1)) - } - - val rightParts = if (isRightSkew) { - val mapStartIndices = getMapStartIndices(right, partitionIndex, rightTargetSize) - if (mapStartIndices.length > 1) { - rightSkewDesc.addPartitionSize(rightSize) - createSkewPartitions(partitionIndex, mapStartIndices, getNumMappers(right)) - } else { - Seq(CoalescedPartitionSpec(partitionIndex, partitionIndex + 1)) - } - } else { - Seq(CoalescedPartitionSpec(partitionIndex, partitionIndex + 1)) + val leftActualSize = leftActualSizes(partitionIndex) + val isLeftSkew = isSkewed(leftActualSize, leftMedSize) && canSplitLeft + val leftPartSpec = left.partitionsWithSizes(partitionIndex)._1 + val isLeftCoalesced = leftPartSpec.startReducerIndex + 1 < leftPartSpec.endReducerIndex + + val rightActualSize = rightActualSizes(partitionIndex) + val isRightSkew = isSkewed(rightActualSize, rightMedSize) && canSplitRight + val rightPartSpec = right.partitionsWithSizes(partitionIndex)._1 + val isRightCoalesced = rightPartSpec.startReducerIndex + 1 < rightPartSpec.endReducerIndex + + // A skewed partition should never be coalesced, but skip it here just to be safe. + val leftParts = if (isLeftSkew && !isLeftCoalesced) { + val reducerId = leftPartSpec.startReducerIndex + val skewSpecs = createSkewPartitionSpecs( + left.shuffleStage.shuffle.shuffleDependency.shuffleId, reducerId, leftTargetSize) + if (skewSpecs.isDefined) { + logDebug(s"Left side partition $partitionIndex " + + s"(${FileUtils.byteCountToDisplaySize(leftActualSize)}) is skewed, " + + s"split it into ${skewSpecs.get.length} parts.") + numSkewedLeft += 1 } + skewSpecs.getOrElse(Seq(leftPartSpec)) + } else { + Seq(leftPartSpec) + } - for { - leftSidePartition <- leftParts - rightSidePartition <- rightParts - } { - leftSidePartitions += leftSidePartition - rightSidePartitions += rightSidePartition + // A skewed partition should never be coalesced, but skip it here just to be safe. + val rightParts = if (isRightSkew && !isRightCoalesced) { + val reducerId = rightPartSpec.startReducerIndex + val skewSpecs = createSkewPartitionSpecs( + right.shuffleStage.shuffle.shuffleDependency.shuffleId, reducerId, rightTargetSize) + if (skewSpecs.isDefined) { + logDebug(s"Right side partition $partitionIndex " + + s"(${FileUtils.byteCountToDisplaySize(rightActualSize)}) is skewed, " + + s"split it into ${skewSpecs.get.length} parts.") + numSkewedRight += 1 } + skewSpecs.getOrElse(Seq(rightPartSpec)) } else { - // Add to `nonSkewPartitionIndices` first, and add real partitions later, in case we can - // coalesce the non-skew partitions. - nonSkewPartitionIndices += partitionIndex - // If this is the last partition, add real partition immediately. - if (partitionIndex == numPartitions - 1) { - createNonSkewPartitions(leftStats, rightStats, nonSkewPartitionIndices).foreach { p => - leftSidePartitions += p - rightSidePartitions += p - } - nonSkewPartitionIndices.clear() - } + Seq(rightPartSpec) + } + + for { + leftSidePartition <- leftParts + rightSidePartition <- rightParts + } { + leftSidePartitions += leftSidePartition + rightSidePartitions += rightSidePartition } } - logDebug("number of skewed partitions: " + - s"left ${leftSkewDesc.numPartitions}, right ${rightSkewDesc.numPartitions}") - if (leftSkewDesc.numPartitions > 0 || rightSkewDesc.numPartitions > 0) { - val newLeft = CustomShuffleReaderExec(left, leftSidePartitions, leftSkewDesc.toString) - val newRight = CustomShuffleReaderExec(right, rightSidePartitions, rightSkewDesc.toString) + logDebug(s"number of skewed partitions: left $numSkewedLeft, right $numSkewedRight") + if (numSkewedLeft > 0 || numSkewedRight > 0) { + val newLeft = CustomShuffleReaderExec(left.shuffleStage, leftSidePartitions) + val newRight = CustomShuffleReaderExec(right.shuffleStage, rightSidePartitions) smj.copy( left = s1.copy(child = newLeft), right = s2.copy(child = newRight), isSkewJoin = true) } else { @@ -252,44 +248,6 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { } } - private def createNonSkewPartitions( - leftStats: MapOutputStatistics, - rightStats: MapOutputStatistics, - nonSkewPartitionIndices: Seq[Int]): Seq[ShufflePartitionSpec] = { - assert(nonSkewPartitionIndices.nonEmpty) - val shouldCoalesce = conf.getConf(SQLConf.COALESCE_PARTITIONS_ENABLED) - if (!shouldCoalesce || nonSkewPartitionIndices.length == 1) { - nonSkewPartitionIndices.map(i => CoalescedPartitionSpec(i, i + 1)) - } else { - // We fall back to Spark default parallelism if the minimum number of coalesced partitions - // is not set, so to avoid perf regressions compared to no coalescing. - val minPartitionNum = conf.getConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM) - .getOrElse(SparkContext.getActive.get.defaultParallelism) - ShufflePartitionsUtil.coalescePartitions( - Array(leftStats, rightStats), - firstPartitionIndex = nonSkewPartitionIndices.head, - // `lastPartitionIndex` is exclusive. - lastPartitionIndex = nonSkewPartitionIndices.last + 1, - advisoryTargetSize = conf.getConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES), - minNumPartitions = minPartitionNum) - } - } - - private def createSkewPartitions( - reducerIndex: Int, - mapStartIndices: Seq[Int], - numMappers: Int): Seq[PartialReducerPartitionSpec] = { - mapStartIndices.indices.map { i => - val startMapIndex = mapStartIndices(i) - val endMapIndex = if (i == mapStartIndices.length - 1) { - numMappers - } else { - mapStartIndices(i + 1) - } - PartialReducerPartitionSpec(reducerIndex, startMapIndex, endMapIndex) - } - } - override def apply(plan: SparkPlan): SparkPlan = { if (!conf.getConf(SQLConf.SKEW_JOIN_ENABLED)) { return plan @@ -304,7 +262,7 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { if (shuffleStages.length == 2) { // When multi table join, there will be too many complex combination to consider. - // Currently we only handle 2 table join like following two use cases. + // Currently we only handle 2 table join like following use case. // SMJ // Sort // Shuffle @@ -328,34 +286,39 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { } } -private class SkewDesc { - private[this] var numSkewedPartitions: Int = 0 - private[this] var totalSize: Long = 0 - private[this] var maxSize: Long = 0 - private[this] var minSize: Long = 0 - - def numPartitions: Int = numSkewedPartitions - - def addPartitionSize(size: Long): Unit = { - if (numSkewedPartitions == 0) { - maxSize = size - minSize = size - } - numSkewedPartitions += 1 - totalSize += size - if (size > maxSize) maxSize = size - if (size < minSize) minSize = size - } +private object ShuffleStage { + def unapply(plan: SparkPlan): Option[ShuffleStageInfo] = plan match { + case s: ShuffleQueryStageExec if s.mapStats.isDefined => + val mapStats = s.mapStats.get + val sizes = mapStats.bytesByPartitionId + val partitions = sizes.zipWithIndex.map { + case (size, i) => CoalescedPartitionSpec(i, i + 1) -> size + } + Some(ShuffleStageInfo(s, mapStats, partitions)) + + case CustomShuffleReaderExec(s: ShuffleQueryStageExec, partitionSpecs) + if s.mapStats.isDefined => + val mapStats = s.mapStats.get + val sizes = mapStats.bytesByPartitionId + val partitions = partitionSpecs.map { + case spec @ CoalescedPartitionSpec(start, end) => + var sum = 0L + var i = start + while (i < end) { + sum += sizes(i) + i += 1 + } + spec -> sum + case other => throw new IllegalArgumentException( + s"Expect CoalescedPartitionSpec but got $other") + } + Some(ShuffleStageInfo(s, mapStats, partitions)) - override def toString: String = { - if (numSkewedPartitions == 0) { - "no skewed partition" - } else { - val maxSizeStr = FileUtils.byteCountToDisplaySize(maxSize) - val minSizeStr = FileUtils.byteCountToDisplaySize(minSize) - val avgSizeStr = FileUtils.byteCountToDisplaySize(totalSize / numSkewedPartitions) - s"$numSkewedPartitions skewed partitions with " + - s"size(max=$maxSizeStr, min=$minSizeStr, avg=$avgSizeStr)" - } + case _ => None } } + +private case class ShuffleStageInfo( + shuffleStage: ShuffleQueryStageExec, + mapStats: MapOutputStatistics, + partitionsWithSizes: Seq[(CoalescedPartitionSpec, Long)]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index d5dc1be63f06e..beaa9721fb5e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -161,6 +161,16 @@ case class ShuffleQueryStageExec( case _ => } } + + /** + * Returns the Option[MapOutputStatistics]. If the shuffle map stage has no partition, + * this method returns None, as there is no map statistics. + */ + def mapStats: Option[MapOutputStatistics] = { + assert(resultOption.isDefined, "ShuffleQueryStageExec should already be ready") + val stats = resultOption.get.asInstanceOf[MapOutputStatistics] + Option(stats) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala index 292df11c23ca1..208cc059d238c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala @@ -28,10 +28,9 @@ object ShufflePartitionsUtil extends Logging { final val MERGED_PARTITION_FACTOR = 1.2 /** - * Coalesce the same range of partitions (`firstPartitionIndex` to `lastPartitionIndex`, the - * start is inclusive and the end is exclusive) from multiple shuffles. This method assumes that - * all the shuffles have the same number of partitions, and the partitions of same index will be - * read together by one task. + * Coalesce the partitions from multiple shuffles. This method assumes that all the shuffles + * have the same number of partitions, and the partitions of same index will be read together + * by one task. * * The strategy used to determine the number of coalesced partitions is described as follows. * To determine the number of coalesced partitions, we have a target size for a coalesced @@ -56,8 +55,6 @@ object ShufflePartitionsUtil extends Logging { */ def coalescePartitions( mapOutputStatistics: Array[MapOutputStatistics], - firstPartitionIndex: Int, - lastPartitionIndex: Int, advisoryTargetSize: Long, minNumPartitions: Int): Seq[ShufflePartitionSpec] = { // If `minNumPartitions` is very large, it is possible that we need to use a value less than @@ -87,11 +84,12 @@ object ShufflePartitionsUtil extends Logging { "There should be only one distinct value of the number of shuffle partitions " + "among registered Exchange operators.") + val numPartitions = distinctNumShufflePartitions.head val partitionSpecs = ArrayBuffer[CoalescedPartitionSpec]() - var latestSplitPoint = firstPartitionIndex + var latestSplitPoint = 0 var coalescedSize = 0L - var i = firstPartitionIndex - while (i < lastPartitionIndex) { + var i = 0 + while (i < numPartitions) { // We calculate the total size of i-th shuffle partitions from all shuffles. var totalSizeOfCurrentPartition = 0L var j = 0 @@ -112,7 +110,7 @@ object ShufflePartitionsUtil extends Logging { } i += 1 } - partitionSpecs += CoalescedPartitionSpec(latestSplitPoint, lastPartitionIndex) + partitionSpecs += CoalescedPartitionSpec(latestSplitPoint, numPartitions) partitionSpecs } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index 81157ca0bfe9b..c047be774d99a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -386,4 +386,12 @@ object CommandUtils extends Logging { private def isDataPath(path: Path, stagingDir: String): Boolean = { !path.getName.startsWith(stagingDir) && DataSourceUtils.isDataPath(path) } + + def uncacheTableOrView(sparkSession: SparkSession, name: String): Unit = { + try { + sparkSession.catalog.uncacheTable(name) + } catch { + case NonFatal(e) => logWarning("Exception when attempting to uncache $name", e) + } + } } 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 c55ff4ffefa02..3dc1d52697714 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 @@ -162,7 +162,8 @@ object SetCommand { } /** - * This command is for resetting SQLConf to the default values. Command that runs + * This command is for resetting SQLConf to the default values. Any configurations that were set + * via [[SetCommand]] will get reset to default value. Command that runs * {{{ * reset; * }}} @@ -170,7 +171,11 @@ object SetCommand { case object ResetCommand extends RunnableCommand with IgnoreCachedData { override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.sessionState.conf.clear() + val conf = sparkSession.sessionState.conf + conf.clear() + sparkSession.sparkContext.conf.getAll.foreach { case (k, v) => + conf.setConfString(k, v) + } Seq.empty[Row] } } 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 6243261d3cd16..2c9ca36c7fcdc 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 @@ -196,11 +196,7 @@ case class AlterTableRenameCommand( // this can happen with Hive tables when the underlying catalog is in-memory. val wasCached = Try(sparkSession.catalog.isCached(oldName.unquotedString)).getOrElse(false) if (wasCached) { - try { - sparkSession.catalog.uncacheTable(oldName.unquotedString) - } catch { - case NonFatal(e) => log.warn(e.toString, e) - } + CommandUtils.uncacheTableOrView(sparkSession, oldName.unquotedString) } // Invalidate the table last, otherwise uncaching the table would load the logical plan // back into the hive metastore cache @@ -230,12 +226,7 @@ case class AlterTableAddColumnsCommand( val catalog = sparkSession.sessionState.catalog val catalogTable = verifyAlterTableAddColumn(sparkSession.sessionState.conf, catalog, table) - try { - sparkSession.catalog.uncacheTable(table.quotedString) - } catch { - case NonFatal(e) => - log.warn(s"Exception when attempting to uncache table ${table.quotedString}", e) - } + CommandUtils.uncacheTableOrView(sparkSession, table.quotedString) catalog.refreshTable(table) SchemaUtils.checkColumnNameDuplication( @@ -508,8 +499,8 @@ case class TruncateTableCommand( var optPermission: Option[FsPermission] = None var optAcls: Option[java.util.List[AclEntry]] = None if (!ignorePermissionAcl) { - val fileStatus = fs.getFileStatus(path) try { + val fileStatus = fs.getFileStatus(path) optPermission = Some(fileStatus.getPermission()) } catch { case NonFatal(_) => // do nothing @@ -1066,6 +1057,42 @@ trait ShowCreateTableCommandBase { protected def concatByMultiLines(iter: Iterable[String]): String = { iter.mkString("(\n ", ",\n ", ")\n") } + + protected def showCreateView(metadata: CatalogTable, builder: StringBuilder): Unit = { + showViewDataColumns(metadata, builder) + showTableComment(metadata, builder) + showViewProperties(metadata, builder) + showViewText(metadata, builder) + } + + private def showViewDataColumns(metadata: CatalogTable, builder: StringBuilder): Unit = { + if (metadata.schema.nonEmpty) { + val viewColumns = metadata.schema.map { f => + val comment = f.getComment() + .map(escapeSingleQuotedString) + .map(" COMMENT '" + _ + "'") + + // view columns shouldn't have data type info + s"${quoteIdentifier(f.name)}${comment.getOrElse("")}" + } + builder ++= concatByMultiLines(viewColumns) + } + } + + private def showViewProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { + val viewProps = metadata.properties.filterKeys(!_.startsWith(CatalogTable.VIEW_PREFIX)) + if (viewProps.nonEmpty) { + val props = viewProps.map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + + builder ++= s"TBLPROPERTIES ${concatByMultiLines(props)}" + } + } + + private def showViewText(metadata: CatalogTable, builder: StringBuilder): Unit = { + builder ++= metadata.viewText.mkString("AS ", "", "\n") + } } /** @@ -1109,10 +1136,6 @@ case class ShowCreateTableCommand(table: TableIdentifier) ) } - if (tableMetadata.tableType == VIEW) { - throw new AnalysisException("Hive view isn't supported by SHOW CREATE TABLE") - } - if ("true".equalsIgnoreCase(tableMetadata.properties.getOrElse("transactional", "false"))) { throw new AnalysisException( "SHOW CREATE TABLE doesn't support transactional Hive table. " + @@ -1120,10 +1143,26 @@ case class ShowCreateTableCommand(table: TableIdentifier) "to show Hive DDL instead.") } - convertTableMetadata(tableMetadata) + if (tableMetadata.tableType == VIEW) { + tableMetadata + } else { + convertTableMetadata(tableMetadata) + } } - val stmt = showCreateDataSourceTable(metadata) + val builder = StringBuilder.newBuilder + + val stmt = if (tableMetadata.tableType == VIEW) { + builder ++= s"CREATE VIEW ${table.quotedString} " + showCreateView(metadata, builder) + + builder.toString() + } else { + builder ++= s"CREATE TABLE ${table.quotedString} " + + showCreateDataSourceTable(metadata, builder) + builder.toString() + } Seq(Row(stmt)) } @@ -1203,18 +1242,13 @@ case class ShowCreateTableCommand(table: TableIdentifier) } } - private def showCreateDataSourceTable(metadata: CatalogTable): String = { - val builder = StringBuilder.newBuilder - - builder ++= s"CREATE TABLE ${table.quotedString} " + private def showCreateDataSourceTable(metadata: CatalogTable, builder: StringBuilder): Unit = { showDataSourceTableDataColumns(metadata, builder) showDataSourceTableOptions(metadata, builder) showDataSourceTableNonDataColumns(metadata, builder) showTableComment(metadata, builder) showTableLocation(metadata, builder) showTableProperties(metadata, builder) - - builder.toString() } } @@ -1273,10 +1307,7 @@ case class ShowCreateTableAsSerdeCommand(table: TableIdentifier) builder ++= s"CREATE$tableTypeString ${table.quotedString}" if (metadata.tableType == VIEW) { - showViewDataColumns(metadata, builder) - showTableComment(metadata, builder) - showViewProperties(metadata, builder) - showViewText(metadata, builder) + showCreateView(metadata, builder) } else { showHiveTableHeader(metadata, builder) showTableComment(metadata, builder) @@ -1289,35 +1320,6 @@ case class ShowCreateTableAsSerdeCommand(table: TableIdentifier) builder.toString() } - private def showViewDataColumns(metadata: CatalogTable, builder: StringBuilder): Unit = { - if (metadata.schema.nonEmpty) { - val viewColumns = metadata.schema.map { f => - val comment = f.getComment() - .map(escapeSingleQuotedString) - .map(" COMMENT '" + _ + "'") - - // view columns shouldn't have data type info - s"${quoteIdentifier(f.name)}${comment.getOrElse("")}" - } - builder ++= concatByMultiLines(viewColumns) - } - } - - private def showViewProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { - val viewProps = metadata.properties.filterKeys(!_.startsWith(CatalogTable.VIEW_PREFIX)) - if (viewProps.nonEmpty) { - val props = viewProps.map { case (key, value) => - s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" - } - - builder ++= s"TBLPROPERTIES ${concatByMultiLines(props)}" - } - } - - private def showViewText(metadata: CatalogTable, builder: StringBuilder): Unit = { - builder ++= metadata.viewText.mkString("AS ", "", "\n") - } - private def showHiveTableHeader(metadata: CatalogTable, builder: StringBuilder): Unit = { val columns = metadata.schema.filterNot { column => metadata.partitionColumnNames.contains(column.name) 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 38481dda428a5..23f1d6c983413 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 @@ -23,10 +23,12 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, UnresolvedFunction, UnresolvedRelation, ViewType} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions.{Alias, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} -import org.apache.spark.sql.types.MetadataBuilder +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper +import org.apache.spark.sql.internal.StaticSQLConf +import org.apache.spark.sql.types.{BooleanType, MetadataBuilder, StringType} import org.apache.spark.sql.util.SchemaUtils /** @@ -108,9 +110,19 @@ case class CreateViewCommand( verifyTemporaryObjectsNotExists(catalog) if (viewType == LocalTempView) { + if (replace && catalog.getTempView(name.table).isDefined) { + logDebug(s"Try to uncache ${name.quotedString} before replacing.") + CommandUtils.uncacheTableOrView(sparkSession, name.quotedString) + } val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) catalog.createTempView(name.table, aliasedPlan, overrideIfExists = replace) } else if (viewType == GlobalTempView) { + if (replace && catalog.getGlobalTempView(name.table).isDefined) { + val db = sparkSession.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) + val globalTempView = TableIdentifier(name.table, Option(db)) + logDebug(s"Try to uncache ${globalTempView.quotedString} before replacing.") + CommandUtils.uncacheTableOrView(sparkSession, globalTempView.quotedString) + } val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) catalog.createGlobalTempView(name.table, aliasedPlan, overrideIfExists = replace) } else if (catalog.tableExists(name)) { @@ -125,6 +137,10 @@ case class CreateViewCommand( val viewIdent = tableMetadata.identifier checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) + // uncache the cached data before replacing an exists view + logDebug(s"Try to uncache ${viewIdent.quotedString} before replacing.") + CommandUtils.uncacheTableOrView(sparkSession, viewIdent.quotedString) + // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` // Nothing we need to retain from the old view, so just drop and create a new one catalog.dropTable(viewIdent, ignoreIfNotExists = false, purge = false) @@ -280,6 +296,40 @@ case class AlterViewAsCommand( } } +/** + * A command for users to get views in the given database. + * If a databaseName is not given, the current database will be used. + * The syntax of using this command in SQL is: + * {{{ + * SHOW VIEWS [(IN|FROM) database_name] [[LIKE] 'identifier_with_wildcards']; + * }}} + */ +case class ShowViewsCommand( + databaseName: String, + tableIdentifierPattern: Option[String]) extends RunnableCommand { + + // The result of SHOW VIEWS has three basic columns: namespace, viewName and isTemporary. + override val output: Seq[Attribute] = Seq( + AttributeReference("namespace", StringType, nullable = false)(), + AttributeReference("viewName", StringType, nullable = false)(), + AttributeReference("isTemporary", BooleanType, nullable = false)()) + + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + + // Show the information of views. + val views = tableIdentifierPattern.map(catalog.listViews(databaseName, _)) + .getOrElse(catalog.listViews(databaseName, "*")) + views.map { tableIdent => + val namespace = tableIdent.database.toArray.quoted + val tableName = tableIdent.table + val isTemp = catalog.isTemporaryTable(tableIdent) + + Row(namespace, tableName, isTemp) + } + } +} + object ViewHelper { import CatalogTable._ 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 1641b660a271d..faf37609ad814 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 @@ -652,10 +652,19 @@ object DataSourceStrategy { */ object PushableColumn { def unapply(e: Expression): Option[String] = { - def helper(e: Expression) = e match { - case a: Attribute => Some(a.name) + val nestedPredicatePushdownEnabled = SQLConf.get.nestedPredicatePushdownEnabled + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper + def helper(e: Expression): Option[Seq[String]] = e match { + case a: Attribute => + if (nestedPredicatePushdownEnabled || !a.name.contains(".")) { + Some(Seq(a.name)) + } else { + None + } + case s: GetStructField if nestedPredicatePushdownEnabled => + helper(s.child).map(_ :+ s.childSchema(s.ordinal).name) case _ => None } - helper(e) + helper(e).map(_.quoted) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala new file mode 100644 index 0000000000000..486f678e49d7d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala @@ -0,0 +1,74 @@ +/* + * 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 + +import java.io.{DataInput, DataOutput, IOException} +import java.sql.Date + +import org.apache.hadoop.hive.serde2.io.DateWritable +import org.apache.hadoop.io.WritableUtils + +import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseJulianToGregorianDays} + +/** + * The class accepts/returns days in Gregorian calendar and rebase them + * via conversion to local date in Julian calendar for dates before 1582-10-15 + * in read/write for backward compatibility with Spark 2.4 and earlier versions. + * + * @param gregorianDays The number of days since the epoch 1970-01-01 in + * Gregorian calendar. + * @param julianDays The number of days since the epoch 1970-01-01 in + * Julian calendar. + */ +class DaysWritable( + var gregorianDays: Int, + var julianDays: Int) + extends DateWritable { + + def this() = this(0, 0) + def this(gregorianDays: Int) = + this(gregorianDays, rebaseGregorianToJulianDays(gregorianDays)) + def this(dateWritable: DateWritable) = { + this( + gregorianDays = dateWritable match { + case daysWritable: DaysWritable => daysWritable.gregorianDays + case dateWritable: DateWritable => + rebaseJulianToGregorianDays(dateWritable.getDays) + }, + julianDays = dateWritable.getDays) + } + + override def getDays: Int = julianDays + override def get(): Date = new Date(DateWritable.daysToMillis(julianDays)) + + override def set(d: Int): Unit = { + gregorianDays = d + julianDays = rebaseGregorianToJulianDays(d) + } + + @throws[IOException] + override def write(out: DataOutput): Unit = { + WritableUtils.writeVInt(out, julianDays) + } + + @throws[IOException] + override def readFields(in: DataInput): Unit = { + julianDays = WritableUtils.readVInt(in) + gregorianDays = rebaseJulianToGregorianDays(julianDays) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala index 50c4f6cd57a96..edb49d3f90ca3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -182,8 +182,7 @@ class DynamicPartitionDataWriter( val partitionName = ScalaUDF( ExternalCatalogUtils.getPartitionPathString _, StringType, - Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId))), - Seq(false, false)) + Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId)))) if (i == 0) Seq(partitionName) else Seq(Literal(Path.SEPARATOR), partitionName) }) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index cac2d6e626120..84160f35540df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -23,6 +23,7 @@ import scala.collection.mutable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.viewfs.ViewFileSystem import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.mapred.{FileInputFormat, JobConf} @@ -313,7 +314,7 @@ object InMemoryFileIndex extends Logging { // to retrieve the file status with the file block location. The reason to still fallback // to listStatus is because the default implementation would potentially throw a // FileNotFoundException which is better handled by doing the lookups manually below. - case _: DistributedFileSystem if !ignoreLocality => + case (_: DistributedFileSystem | _: ViewFileSystem) if !ignoreLocality => val remoteIter = fs.listLocatedStatus(path) new Iterator[LocatedFileStatus]() { def next(): LocatedFileStatus = remoteIter.next diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index b0ec24e9b1a3d..292ac6db04baf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -131,7 +131,10 @@ object PartitioningUtils { } val dateFormatter = DateFormatter(zoneId) - val timestampFormatter = TimestampFormatter(timestampPartitionPattern, zoneId) + val timestampFormatter = TimestampFormatter( + timestampPartitionPattern, + zoneId, + needVarLengthSecondFraction = true) // First, we need to parse every partition's path and see if we can find partition values. val (partitionValues, optDiscoveredBasePaths) = paths.map { path => parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala index ccaff0d6ca7d4..c864f1f52fcce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala @@ -28,6 +28,9 @@ import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions * the parameters. */ private[jdbc] trait ConnectionProvider { + /** + * Opens connection toward the database. + */ def getConnection(): Connection } @@ -43,6 +46,10 @@ private[jdbc] object ConnectionProvider extends Logging { logDebug("Postgres connection provider found") new PostgresConnectionProvider(driver, options) + case MariaDBConnectionProvider.driverClass => + logDebug("MariaDB connection provider found") + new MariaDBConnectionProvider(driver, options) + case _ => throw new IllegalArgumentException(s"Driver ${options.driverClass} does not support " + "Kerberos authentication") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala new file mode 100644 index 0000000000000..eb2f0f78022ba --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala @@ -0,0 +1,54 @@ +/* + * 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.jdbc.connection + +import java.sql.Driver +import javax.security.auth.login.Configuration + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions + +private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptions) + extends SecureConnectionProvider(driver, options) { + override val appEntry: String = { + "Krb5ConnectorContext" + } + + override def setAuthenticationConfigIfNeeded(): Unit = { + val parent = Configuration.getConfiguration + val configEntry = parent.getAppConfigurationEntry(appEntry) + /** + * Couple of things to mention here: + * 1. MariaDB doesn't support JAAS application name configuration + * 2. MariaDB sets a default JAAS config if "java.security.auth.login.config" is not set + */ + val entryUsesKeytab = configEntry != null && + configEntry.exists(_.getOptions().get("useKeyTab") == "true") + if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) { + val config = new SecureConnectionProvider.JDBCConfiguration( + parent, appEntry, options.keytab, options.principal) + logDebug("Adding database specific security configuration") + Configuration.setConfiguration(config) + } + } +} + +private[sql] object MariaDBConnectionProvider { + val driverClass = "org.mariadb.jdbc.Driver" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala index e793c4dfd780e..14911fc75ebc1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala @@ -17,66 +17,32 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection -import java.sql.{Connection, Driver} +import java.sql.Driver import java.util.Properties -import javax.security.auth.login.{AppConfigurationEntry, Configuration} - -import scala.collection.JavaConverters._ +import javax.security.auth.login.Configuration import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions -import org.apache.spark.sql.execution.datasources.jdbc.connection.PostgresConnectionProvider.PGJDBCConfiguration -import org.apache.spark.util.SecurityUtils private[jdbc] class PostgresConnectionProvider(driver: Driver, options: JDBCOptions) - extends BasicConnectionProvider(driver, options) { - val appEntry: String = { + extends SecureConnectionProvider(driver, options) { + override val appEntry: String = { val parseURL = driver.getClass.getMethod("parseURL", classOf[String], classOf[Properties]) val properties = parseURL.invoke(driver, options.url, null).asInstanceOf[Properties] properties.getProperty("jaasApplicationName", "pgjdbc") } - def setAuthenticationConfigIfNeeded(): Unit = { + override def setAuthenticationConfigIfNeeded(): Unit = { val parent = Configuration.getConfiguration val configEntry = parent.getAppConfigurationEntry(appEntry) if (configEntry == null || configEntry.isEmpty) { - val config = new PGJDBCConfiguration(parent, appEntry, options.keytab, options.principal) + val config = new SecureConnectionProvider.JDBCConfiguration( + parent, appEntry, options.keytab, options.principal) + logDebug("Adding database specific security configuration") Configuration.setConfiguration(config) } } - - override def getConnection(): Connection = { - setAuthenticationConfigIfNeeded() - super.getConnection() - } } private[sql] object PostgresConnectionProvider { - class PGJDBCConfiguration( - parent: Configuration, - appEntry: String, - keytab: String, - principal: String) extends Configuration { - private val entry = - new AppConfigurationEntry( - SecurityUtils.getKrb5LoginModuleName(), - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, - Map[String, Object]( - "useTicketCache" -> "false", - "useKeyTab" -> "true", - "keyTab" -> keytab, - "principal" -> principal, - "debug" -> "true" - ).asJava - ) - - override def getAppConfigurationEntry(name: String): Array[AppConfigurationEntry] = { - if (name.equals(appEntry)) { - Array(entry) - } else { - parent.getAppConfigurationEntry(name) - } - } - } - val driverClass = "org.postgresql.Driver" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala new file mode 100644 index 0000000000000..ff192d71e6f33 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala @@ -0,0 +1,75 @@ +/* + * 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.jdbc.connection + +import java.sql.{Connection, Driver} +import javax.security.auth.login.{AppConfigurationEntry, Configuration} + +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions +import org.apache.spark.util.SecurityUtils + +private[jdbc] abstract class SecureConnectionProvider(driver: Driver, options: JDBCOptions) + extends BasicConnectionProvider(driver, options) with Logging { + override def getConnection(): Connection = { + setAuthenticationConfigIfNeeded() + super.getConnection() + } + + /** + * Returns JAAS application name. This is sometimes configurable on the JDBC driver level. + */ + val appEntry: String + + /** + * Sets database specific authentication configuration when needed. If configuration already set + * then later calls must be no op. + */ + def setAuthenticationConfigIfNeeded(): Unit +} + +object SecureConnectionProvider { + class JDBCConfiguration( + parent: Configuration, + appEntry: String, + keytab: String, + principal: String) extends Configuration { + val entry = + new AppConfigurationEntry( + SecurityUtils.getKrb5LoginModuleName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + Map[String, Object]( + "useTicketCache" -> "false", + "useKeyTab" -> "true", + "keyTab" -> keytab, + "principal" -> principal, + "debug" -> "true" + ).asJava + ) + + override def getAppConfigurationEntry(name: String): Array[AppConfigurationEntry] = { + if (name.equals(appEntry)) { + Array(entry) + } else { + parent.getAppConfigurationEntry(name) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala index 6d52d40d6dd03..4ab009c6bd014 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala @@ -23,6 +23,7 @@ import org.apache.orc.mapred.{OrcList, OrcMap, OrcStruct, OrcTimestamp} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData} import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.RebaseDateTime.rebaseJulianToGregorianDays import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -108,7 +109,7 @@ class OrcDeserializer( updater.set(ordinal, bytes) case DateType => (ordinal, value) => - updater.setInt(ordinal, DateTimeUtils.fromJavaDate(OrcShimUtils.getSqlDate(value))) + updater.setInt(ordinal, OrcShimUtils.getGregorianDays(value)) case TimestampType => (ordinal, value) => updater.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(value.asInstanceOf[OrcTimestamp])) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 07065018343cf..f206f59dacdc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -27,7 +27,7 @@ import scala.collection.JavaConverters.asScalaBufferConverter import org.apache.parquet.filter2.predicate._ import org.apache.parquet.filter2.predicate.SparkFilterApi._ import org.apache.parquet.io.api.Binary -import org.apache.parquet.schema.{DecimalMetadata, MessageType, OriginalType, PrimitiveComparator} +import org.apache.parquet.schema.{DecimalMetadata, GroupType, MessageType, OriginalType, PrimitiveComparator, PrimitiveType, Type} import org.apache.parquet.schema.OriginalType._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ @@ -49,15 +49,35 @@ class ParquetFilters( pushDownInFilterThreshold: Int, caseSensitive: Boolean) { // A map which contains parquet field name and data type, if predicate push down applies. - private val nameToParquetField : Map[String, ParquetField] = { - // Here we don't flatten the fields in the nested schema but just look up through - // root fields. Currently, accessing to nested fields does not push down filters - // and it does not support to create filters for them. - val primitiveFields = - schema.getFields.asScala.filter(_.isPrimitive).map(_.asPrimitiveType()).map { f => - f.getName -> ParquetField(f.getName, - ParquetSchemaType(f.getOriginalType, - f.getPrimitiveTypeName, f.getTypeLength, f.getDecimalMetadata)) + // + // Each key in `nameToParquetField` represents a column; `dots` are used as separators for + // nested columns. If any part of the names contains `dots`, it is quoted to avoid confusion. + // See `org.apache.spark.sql.connector.catalog.quote` for implementation details. + private val nameToParquetField : Map[String, ParquetPrimitiveField] = { + // Recursively traverse the parquet schema to get primitive fields that can be pushed-down. + // `parentFieldNames` is used to keep track of the current nested level when traversing. + def getPrimitiveFields( + fields: Seq[Type], + parentFieldNames: Array[String] = Array.empty): Seq[ParquetPrimitiveField] = { + fields.flatMap { + case p: PrimitiveType => + Some(ParquetPrimitiveField(fieldNames = parentFieldNames :+ p.getName, + fieldType = ParquetSchemaType(p.getOriginalType, + p.getPrimitiveTypeName, p.getTypeLength, p.getDecimalMetadata))) + // Note that when g is a `Struct`, `g.getOriginalType` is `null`. + // When g is a `Map`, `g.getOriginalType` is `MAP`. + // When g is a `List`, `g.getOriginalType` is `LIST`. + case g: GroupType if g.getOriginalType == null => + getPrimitiveFields(g.getFields.asScala, parentFieldNames :+ g.getName) + // Parquet only supports push-down for primitive types; as a result, Map and List types + // are removed. + case _ => None + } + } + + val primitiveFields = getPrimitiveFields(schema.getFields.asScala).map { field => + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper + (field.fieldNames.toSeq.quoted, field) } if (caseSensitive) { primitiveFields.toMap @@ -75,13 +95,13 @@ class ParquetFilters( } /** - * Holds a single field information stored in the underlying parquet file. + * Holds a single primitive field information stored in the underlying parquet file. * - * @param fieldName field name in parquet file + * @param fieldNames a field name as an array of string multi-identifier in parquet file * @param fieldType field type related info in parquet file */ - private case class ParquetField( - fieldName: String, + private case class ParquetPrimitiveField( + fieldNames: Array[String], fieldType: ParquetSchemaType) private case class ParquetSchemaType( @@ -472,13 +492,8 @@ class ParquetFilters( case _ => false } - // Parquet does not allow dots in the column name because dots are used as a column path - // delimiter. Since Parquet 1.8.2 (PARQUET-389), Parquet accepts the filter predicates - // with missing columns. The incorrect results could be got from Parquet when we push down - // filters for the column having dots in the names. Thus, we do not push down such filters. - // See SPARK-20364. private def canMakeFilterOn(name: String, value: Any): Boolean = { - nameToParquetField.contains(name) && !name.contains(".") && valueCanMakeFilterOn(name, value) + nameToParquetField.contains(name) && valueCanMakeFilterOn(name, value) } /** @@ -509,38 +524,38 @@ class ParquetFilters( predicate match { case sources.IsNull(name) if canMakeFilterOn(name, null) => makeEq.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), null)) + .map(_(nameToParquetField(name).fieldNames, null)) case sources.IsNotNull(name) if canMakeFilterOn(name, null) => makeNotEq.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), null)) + .map(_(nameToParquetField(name).fieldNames, null)) case sources.EqualTo(name, value) if canMakeFilterOn(name, value) => makeEq.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), value)) + .map(_(nameToParquetField(name).fieldNames, value)) case sources.Not(sources.EqualTo(name, value)) if canMakeFilterOn(name, value) => makeNotEq.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), value)) + .map(_(nameToParquetField(name).fieldNames, value)) case sources.EqualNullSafe(name, value) if canMakeFilterOn(name, value) => makeEq.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), value)) + .map(_(nameToParquetField(name).fieldNames, value)) case sources.Not(sources.EqualNullSafe(name, value)) if canMakeFilterOn(name, value) => makeNotEq.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), value)) + .map(_(nameToParquetField(name).fieldNames, value)) case sources.LessThan(name, value) if canMakeFilterOn(name, value) => makeLt.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), value)) + .map(_(nameToParquetField(name).fieldNames, value)) case sources.LessThanOrEqual(name, value) if canMakeFilterOn(name, value) => makeLtEq.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), value)) + .map(_(nameToParquetField(name).fieldNames, value)) case sources.GreaterThan(name, value) if canMakeFilterOn(name, value) => makeGt.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), value)) + .map(_(nameToParquetField(name).fieldNames, value)) case sources.GreaterThanOrEqual(name, value) if canMakeFilterOn(name, value) => makeGtEq.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), value)) + .map(_(nameToParquetField(name).fieldNames, value)) case sources.And(lhs, rhs) => // At here, it is not safe to just convert one side and remove the other side @@ -591,13 +606,13 @@ class ParquetFilters( && values.distinct.length <= pushDownInFilterThreshold => values.distinct.flatMap { v => makeEq.lift(nameToParquetField(name).fieldType) - .map(_(Array(nameToParquetField(name).fieldName), v)) + .map(_(nameToParquetField(name).fieldNames, v)) }.reduceLeftOption(FilterApi.or) case sources.StringStartsWith(name, prefix) if pushDownStartWith && canMakeFilterOn(name, prefix) => Option(prefix).map { v => - FilterApi.userDefined(binaryColumn(Array(nameToParquetField(name).fieldName)), + FilterApi.userDefined(binaryColumn(nameToParquetField(name).fieldNames), new UserDefinedPredicate[Binary] with Serializable { private val strToBinary = Binary.fromReusedByteArray(v.getBytes) private val size = strToBinary.length diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index ff686d024f111..6072db102fe3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -33,8 +33,10 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.{BINARY, DOUBLE import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, CaseInsensitiveMap, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.util.DateTimeUtils.SQLTimestamp +import org.apache.spark.sql.catalyst.util.RebaseDateTime._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -129,6 +131,9 @@ private[parquet] class ParquetRowConverter( updater: ParentContainerUpdater) extends ParquetGroupConverter(updater) with Logging { + // Enable rebasing date/timestamp from Julian to Proleptic Gregorian calendar + private val rebaseDateTime = SQLConf.get.parquetRebaseDateTimeInReadEnabled + assert( parquetType.getFieldCount <= catalystType.length, s"""Field count of the Parquet schema is greater than the field count of the Catalyst schema: @@ -178,8 +183,15 @@ private[parquet] class ParquetRowConverter( // Converters for each field. private[this] val fieldConverters: Array[Converter with HasParentContainerUpdater] = { + // (SPARK-31116) Use case insensitive map if spark.sql.caseSensitive is false + // to prevent throwing IllegalArgumentException when searching catalyst type's field index + val catalystFieldNameToIndex = if (SQLConf.get.caseSensitiveAnalysis) { + catalystType.fieldNames.zipWithIndex.toMap + } else { + CaseInsensitiveMap(catalystType.fieldNames.zipWithIndex.toMap) + } parquetType.getFields.asScala.map { parquetField => - val fieldIndex = catalystType.fieldIndex(parquetField.getName) + val fieldIndex = catalystFieldNameToIndex(parquetField.getName) val catalystField = catalystType(fieldIndex) // Converted field value should be set to the `fieldIndex`-th cell of `currentRow` newConverter(parquetField, catalystField.dataType, new RowUpdater(currentRow, fieldIndex)) @@ -263,16 +275,35 @@ private[parquet] class ParquetRowConverter( new ParquetStringConverter(updater) case TimestampType if parquetType.getOriginalType == OriginalType.TIMESTAMP_MICROS => - new ParquetPrimitiveConverter(updater) { - override def addLong(value: Long): Unit = { - updater.setLong(value) + if (rebaseDateTime) { + new ParquetPrimitiveConverter(updater) { + override def addLong(value: Long): Unit = { + val rebased = rebaseJulianToGregorianMicros(value) + updater.setLong(rebased) + } + } + } else { + new ParquetPrimitiveConverter(updater) { + override def addLong(value: Long): Unit = { + updater.setLong(value) + } } } case TimestampType if parquetType.getOriginalType == OriginalType.TIMESTAMP_MILLIS => - new ParquetPrimitiveConverter(updater) { - override def addLong(value: Long): Unit = { - updater.setLong(DateTimeUtils.millisToMicros(value)) + if (rebaseDateTime) { + new ParquetPrimitiveConverter(updater) { + override def addLong(value: Long): Unit = { + val micros = DateTimeUtils.millisToMicros(value) + val rebased = rebaseJulianToGregorianMicros(micros) + updater.setLong(rebased) + } + } + } else { + new ParquetPrimitiveConverter(updater) { + override def addLong(value: Long): Unit = { + updater.setLong(DateTimeUtils.millisToMicros(value)) + } } } @@ -297,10 +328,17 @@ private[parquet] class ParquetRowConverter( } case DateType => - new ParquetPrimitiveConverter(updater) { - override def addInt(value: Int): Unit = { - // DateType is not specialized in `SpecificMutableRow`, have to box it here. - updater.set(value.asInstanceOf[DateType#InternalType]) + if (rebaseDateTime) { + new ParquetPrimitiveConverter(updater) { + override def addInt(value: Int): Unit = { + updater.set(rebaseJulianToGregorianDays(value)) + } + } + } else { + new ParquetPrimitiveConverter(updater) { + override def addInt(value: Int): Unit = { + updater.set(value) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index bfa33ea237395..7317a250060b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.SPARK_VERSION_METADATA_KEY import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecializedGetters import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.RebaseDateTime._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -77,6 +78,10 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { private val decimalBuffer = new Array[Byte](Decimal.minBytesForPrecision(DecimalType.MAX_PRECISION)) + // Whether to rebase datetimes from Gregorian to Julian calendar in write + private val rebaseDateTime: Boolean = + SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_WRITE) + override def init(configuration: Configuration): WriteContext = { val schemaString = configuration.get(ParquetWriteSupport.SPARK_ROW_SCHEMA) this.schema = StructType.fromString(schemaString) @@ -147,6 +152,11 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { (row: SpecializedGetters, ordinal: Int) => recordConsumer.addInteger(row.getShort(ordinal)) + case DateType if rebaseDateTime => + (row: SpecializedGetters, ordinal: Int) => + val rebasedDays = rebaseGregorianToJulianDays(row.getInt(ordinal)) + recordConsumer.addInteger(rebasedDays) + case IntegerType | DateType => (row: SpecializedGetters, ordinal: Int) => recordConsumer.addInteger(row.getInt(ordinal)) @@ -177,10 +187,21 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { buf.order(ByteOrder.LITTLE_ENDIAN).putLong(timeOfDayNanos).putInt(julianDay) recordConsumer.addBinary(Binary.fromReusedByteArray(timestampBuffer)) + case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS if rebaseDateTime => + (row: SpecializedGetters, ordinal: Int) => + val rebasedMicros = rebaseGregorianToJulianMicros(row.getLong(ordinal)) + recordConsumer.addLong(rebasedMicros) + case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS => (row: SpecializedGetters, ordinal: Int) => recordConsumer.addLong(row.getLong(ordinal)) + case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS if rebaseDateTime => + (row: SpecializedGetters, ordinal: Int) => + val rebasedMicros = rebaseGregorianToJulianMicros(row.getLong(ordinal)) + val millis = DateTimeUtils.microsToMillis(rebasedMicros) + recordConsumer.addLong(millis) + case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS => (row: SpecializedGetters, ordinal: Int) => val millis = DateTimeUtils.microsToMillis(row.getLong(ordinal)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala index d519832c57501..cd62ee7814bf2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala @@ -16,7 +16,6 @@ */ package org.apache.spark.sql.execution.datasources.v2 -import java.io.IOException import java.util.UUID import scala.collection.JavaConverters._ @@ -27,7 +26,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.spark.internal.io.FileCommitProtocol -import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, WriteBuilder} @@ -46,12 +45,6 @@ abstract class FileWriteBuilder( private val schema = info.schema() private val queryId = info.queryId() private val options = info.options() - private var mode: SaveMode = _ - - def mode(mode: SaveMode): WriteBuilder = { - this.mode = mode - this - } override def buildForBatch(): BatchWrite = { val sparkSession = SparkSession.active @@ -68,26 +61,8 @@ abstract class FileWriteBuilder( lazy val description = createWriteJobDescription(sparkSession, hadoopConf, job, paths.head, options.asScala.toMap) - val fs = path.getFileSystem(hadoopConf) - mode match { - case SaveMode.ErrorIfExists if fs.exists(path) => - val qualifiedOutputPath = path.makeQualified(fs.getUri, fs.getWorkingDirectory) - throw new AnalysisException(s"path $qualifiedOutputPath already exists.") - - case SaveMode.Ignore if fs.exists(path) => - null - - case SaveMode.Overwrite => - if (fs.exists(path) && !committer.deleteWithJob(fs, path, true)) { - throw new IOException(s"Unable to clear directory $path prior to writing to it") - } - committer.setupJob(job) - new FileBatchWrite(job, description, committer) - - case _ => - committer.setupJob(job) - new FileBatchWrite(job, description, committer) - } + committer.setupJob(job) + new FileBatchWrite(job, description, committer) } /** @@ -104,7 +79,6 @@ abstract class FileWriteBuilder( private def validateInputs(caseSensitiveAnalysis: Boolean): Unit = { assert(schema != null, "Missing input data schema") assert(queryId != null, "Missing query ID") - assert(mode != null, "Missing save mode") if (paths.length != 1) { throw new IllegalArgumentException("Expected exactly one path to be specified, but " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala index fe3ab8023db6f..6f968481cb7cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchem import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.connector.catalog.SupportsNamespaces +import org.apache.spark.sql.execution.LeafExecNode /** * Physical plan node for showing namespaces. @@ -33,8 +34,7 @@ case class ShowNamespacesExec( output: Seq[Attribute], catalog: SupportsNamespaces, namespace: Seq[String], - pattern: Option[String]) - extends V2CommandExec { + pattern: Option[String]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { val namespaces = if (namespace.nonEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 995b00871fc2a..c740e0d370dfd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchem import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.execution.LeafExecNode /** * Physical plan node for showing tables. @@ -33,8 +34,7 @@ case class ShowTablesExec( output: Seq[Attribute], catalog: TableCatalog, namespace: Seq[String], - pattern: Option[String]) - extends V2CommandExec { + pattern: Option[String]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() val encoder = RowEncoder(schema).resolveAndBind() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala index f97300025400d..7502a8750aee4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala @@ -41,7 +41,7 @@ case class AppendDataExecV1( writeOptions: CaseInsensitiveStringMap, plan: LogicalPlan) extends V1FallbackWriters { - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { writeWithV1(newWriteBuilder().buildForV1Write()) } } @@ -67,7 +67,7 @@ case class OverwriteByExpressionExecV1( filters.length == 1 && filters(0).isInstanceOf[AlwaysTrue] } - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { newWriteBuilder() match { case builder: SupportsTruncate if isTruncate(deleteWhere) => writeWithV1(builder.truncate().asV1Builder.buildForV1Write()) @@ -82,7 +82,7 @@ case class OverwriteByExpressionExecV1( } /** Some helper interfaces that use V2 write semantics through the V1 writer interface. */ -sealed trait V1FallbackWriters extends SupportsV1Write { +sealed trait V1FallbackWriters extends V2CommandExec with SupportsV1Write { override def output: Seq[Attribute] = Nil override final def children: Seq[SparkPlan] = Nil @@ -115,8 +115,8 @@ trait SupportsV1Write extends SparkPlan { // TODO: We should be able to work on SparkPlans at this point. def plan: LogicalPlan - protected def writeWithV1(relation: InsertableRelation): RDD[InternalRow] = { + protected def writeWithV1(relation: InsertableRelation): Seq[InternalRow] = { relation.insert(Dataset.ofRows(sqlContext.sparkSession, plan), overwrite = false) - sparkContext.emptyRDD + Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala index a1f685d47a346..4be4a6b30edcd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala @@ -19,13 +19,13 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.SparkPlan /** * A physical operator that executes run() and saves the result to prevent multiple executions. * Any V2 commands that do not require triggering a spark job should extend this class. */ -abstract class V2CommandExec extends LeafExecNode { +abstract class V2CommandExec extends SparkPlan { /** * Abstract method that each concrete command needs to implement to compute the result. @@ -53,4 +53,6 @@ abstract class V2CommandExec extends LeafExecNode { protected override def doExecute(): RDD[InternalRow] = { sqlContext.sparkContext.parallelize(result, 1) } + + override def children: Seq[SparkPlan] = Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index e360a9e656a16..616e18ee85a6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -70,10 +70,10 @@ case class CreateTableAsSelectExec( import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { if (catalog.tableExists(ident)) { if (ifNotExists) { - return sparkContext.parallelize(Seq.empty, 1) + return Nil } throw new TableAlreadyExistsException(ident) @@ -125,10 +125,10 @@ case class AtomicCreateTableAsSelectExec( writeOptions: CaseInsensitiveStringMap, ifNotExists: Boolean) extends AtomicTableWriteExec { - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { if (catalog.tableExists(ident)) { if (ifNotExists) { - return sparkContext.parallelize(Seq.empty, 1) + return Nil } throw new TableAlreadyExistsException(ident) @@ -161,7 +161,7 @@ case class ReplaceTableAsSelectExec( import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { // Note that this operation is potentially unsafe, but these are the strict semantics of // RTAS if the catalog does not support atomic operations. // @@ -225,7 +225,7 @@ case class AtomicReplaceTableAsSelectExec( writeOptions: CaseInsensitiveStringMap, orCreate: Boolean) extends AtomicTableWriteExec { - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { val schema = query.schema.asNullable val staged = if (orCreate) { catalog.stageCreateOrReplace( @@ -255,7 +255,7 @@ case class AppendDataExec( writeOptions: CaseInsensitiveStringMap, query: SparkPlan) extends V2TableWriteExec with BatchWriteHelper { - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { writeWithV2(newWriteBuilder().buildForBatch()) } } @@ -280,7 +280,7 @@ case class OverwriteByExpressionExec( filters.length == 1 && filters(0).isInstanceOf[AlwaysTrue] } - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { newWriteBuilder() match { case builder: SupportsTruncate if isTruncate(deleteWhere) => writeWithV2(builder.truncate().buildForBatch()) @@ -308,7 +308,7 @@ case class OverwritePartitionsDynamicExec( writeOptions: CaseInsensitiveStringMap, query: SparkPlan) extends V2TableWriteExec with BatchWriteHelper { - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { newWriteBuilder() match { case builder: SupportsDynamicOverwrite => writeWithV2(builder.overwriteDynamicPartitions().buildForBatch()) @@ -325,7 +325,7 @@ case class WriteToDataSourceV2Exec( def writeOptions: CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty() - override protected def doExecute(): RDD[InternalRow] = { + override protected def run(): Seq[InternalRow] = { writeWithV2(batchWrite) } } @@ -350,7 +350,7 @@ trait BatchWriteHelper { /** * The base physical plan for writing data into data source v2. */ -trait V2TableWriteExec extends UnaryExecNode { +trait V2TableWriteExec extends V2CommandExec with UnaryExecNode { def query: SparkPlan var commitProgress: Option[StreamWriterCommitProgress] = None @@ -358,7 +358,7 @@ trait V2TableWriteExec extends UnaryExecNode { override def child: SparkPlan = query override def output: Seq[Attribute] = Nil - protected def writeWithV2(batchWrite: BatchWrite): RDD[InternalRow] = { + protected def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = { val rdd: RDD[InternalRow] = { val tempRdd = query.execute() // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single @@ -415,7 +415,7 @@ trait V2TableWriteExec extends UnaryExecNode { } } - sparkContext.emptyRDD + Nil } } @@ -485,7 +485,7 @@ private[v2] trait AtomicTableWriteExec extends V2TableWriteExec with SupportsV1W protected def writeToStagedTable( stagedTable: StagedTable, writeOptions: CaseInsensitiveStringMap, - ident: Identifier): RDD[InternalRow] = { + ident: Identifier): Seq[InternalRow] = { Utils.tryWithSafeFinallyAndFailureCallbacks({ stagedTable match { case table: SupportsWrite => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 1421ffd8b6de4..9f40f5faa2e99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -22,6 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.quoteIfNeeded import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.orc.OrcFilters @@ -59,8 +60,10 @@ case class OrcScanBuilder( // changed `hadoopConf` in executors. OrcInputFormat.setSearchArgument(hadoopConf, f, schema.fieldNames) } - val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap - _pushedFilters = OrcFilters.convertibleFilters(schema, dataTypeMap, filters).toArray + val dataTypeMap = schema.map(f => quoteIfNeeded(f.name) -> f.dataType).toMap + // TODO (SPARK-25557): ORC doesn't support nested predicate pushdown, so they are removed. + val newFilters = filters.filter(!_.containsNestedColumn) + _pushedFilters = OrcFilters.convertibleFilters(schema, dataTypeMap, newFilters).toArray } filters } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 6a57ef2cafe23..6c40104e52a5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -113,7 +113,7 @@ package object debug { s case s => s } - codegenSubtrees.toSeq.map { subtree => + codegenSubtrees.toSeq.sortBy(_.codegenStageId).map { subtree => val (_, source) = subtree.doCodeGen() val codeStats = try { CodeGenerator.compile(source)._2 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index 65e6b7c2f0fba..f69da86e258df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -34,7 +34,8 @@ import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.joins.HashedRelation import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} -import org.apache.spark.util.{SparkFatalException, ThreadUtils, Utils} +import org.apache.spark.unsafe.map.BytesToBytesMap +import org.apache.spark.util.{SparkFatalException, ThreadUtils} /** * A [[BroadcastExchangeExec]] collects, transforms and finally broadcasts the result of @@ -43,6 +44,7 @@ import org.apache.spark.util.{SparkFatalException, ThreadUtils, Utils} case class BroadcastExchangeExec( mode: BroadcastMode, child: SparkPlan) extends Exchange { + import BroadcastExchangeExec._ private[sql] val runId: UUID = UUID.randomUUID @@ -82,9 +84,9 @@ case class BroadcastExchangeExec( val beforeCollect = System.nanoTime() // Use executeCollect/executeCollectIterator to avoid conversion to Scala types val (numRows, input) = child.executeCollectIterator() - if (numRows >= 512000000) { + if (numRows >= MAX_BROADCAST_TABLE_ROWS) { throw new SparkException( - s"Cannot broadcast the table with 512 million or more rows: $numRows rows") + s"Cannot broadcast the table over $MAX_BROADCAST_TABLE_ROWS rows: $numRows rows") } val beforeBuild = System.nanoTime() @@ -104,7 +106,7 @@ case class BroadcastExchangeExec( } longMetric("dataSize") += dataSize - if (dataSize >= (8L << 30)) { + if (dataSize >= MAX_BROADCAST_TABLE_BYTES) { throw new SparkException( s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB") } @@ -173,6 +175,13 @@ case class BroadcastExchangeExec( } object BroadcastExchangeExec { + // Since the maximum number of keys that BytesToBytesMap supports is 1 << 29, + // and only 70% of the slots can be used before growing in HashedRelation, + // here the limitation should not be over 341 million. + val MAX_BROADCAST_TABLE_ROWS = (BytesToBytesMap.MAX_CAPACITY / 1.5).toLong + + val MAX_BROADCAST_TABLE_BYTES = 8L << 30 + private[execution] val executionContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", SQLConf.get.getConf(StaticSQLConf.BROADCAST_EXCHANGE_MAX_THREAD_THRESHOLD))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 65aabe004d75b..92d217983f4c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -116,26 +116,23 @@ object SQLMetrics { // data size total (min, med, max): // 100GB (100MB, 1GB, 10GB) val acc = new SQLMetric(SIZE_METRIC, -1) - acc.register(sc, name = Some(s"$name total (min, med, max (stageId (attemptId): taskId))"), - countFailedValues = false) + acc.register(sc, name = Some(name), countFailedValues = false) acc } def createTimingMetric(sc: SparkContext, name: String): SQLMetric = { // The final result of this metric in physical operator UI may looks like: - // duration(min, med, max): + // duration total (min, med, max): // 5s (800ms, 1s, 2s) val acc = new SQLMetric(TIMING_METRIC, -1) - acc.register(sc, name = Some(s"$name total (min, med, max (stageId (attemptId): taskId))"), - countFailedValues = false) + acc.register(sc, name = Some(name), countFailedValues = false) acc } def createNanoTimingMetric(sc: SparkContext, name: String): SQLMetric = { // Same with createTimingMetric, just normalize the unit of time to millisecond. val acc = new SQLMetric(NS_TIMING_METRIC, -1) - acc.register(sc, name = Some(s"$name total (min, med, max (stageId (attemptId): taskId))"), - countFailedValues = false) + acc.register(sc, name = Some(name), countFailedValues = false) acc } @@ -150,8 +147,7 @@ object SQLMetrics { // probe avg (min, med, max): // (1.2, 2.2, 6.3) val acc = new SQLMetric(AVERAGE_METRIC) - acc.register(sc, name = Some(s"$name (min, med, max (stageId (attemptId): taskId))"), - countFailedValues = false) + acc.register(sc, name = Some(name), countFailedValues = false) acc } @@ -164,34 +160,38 @@ object SQLMetrics { metricsType != SUM_METRIC } + private val METRICS_NAME_SUFFIX = "(min, med, max (stageId: taskId))" + /** * A function that defines how we aggregate the final accumulator results among all tasks, * and represent it in string for a SQL physical operator. */ def stringValue(metricsType: String, values: Array[Long], maxMetrics: Array[Long]): String = { - // stringMetric = "(driver)" OR (stage $stageId (attempt $attemptId): task $taskId)) - val stringMetric = if (maxMetrics.isEmpty) { + // taskInfo = "(driver)" OR (stage ${stageId}.${attemptId}: task $taskId) + val taskInfo = if (maxMetrics.isEmpty) { "(driver)" } else { - s"(stage ${maxMetrics(1)} (attempt ${maxMetrics(2)}): task ${maxMetrics(3)})" + s"(stage ${maxMetrics(1)}.${maxMetrics(2)}: task ${maxMetrics(3)})" } if (metricsType == SUM_METRIC) { val numberFormat = NumberFormat.getIntegerInstance(Locale.US) numberFormat.format(values.sum) } else if (metricsType == AVERAGE_METRIC) { val validValues = values.filter(_ > 0) - val Seq(min, med, max) = { - val metric = if (validValues.isEmpty) { - val zeros = Seq.fill(3)(0L) - zeros.map(v => toNumberFormat(v)) - } else { + // When there are only 1 metrics value (or None), no need to display max/min/median. This is + // common for driver-side SQL metrics. + if (validValues.length <= 1) { + toNumberFormat(validValues.headOption.getOrElse(0)) + } else { + val Seq(min, med, max) = { Arrays.sort(validValues) - Seq(toNumberFormat(validValues(0)), toNumberFormat(validValues(validValues.length / 2)), - s"${toNumberFormat(validValues(validValues.length - 1))} $stringMetric") + Seq( + toNumberFormat(validValues(0)), + toNumberFormat(validValues(validValues.length / 2)), + toNumberFormat(validValues(validValues.length - 1))) } - metric + s"$METRICS_NAME_SUFFIX:\n($min, $med, $max $taskInfo)" } - s"\n($min, $med, $max)" } else { val strFormat: Long => String = if (metricsType == SIZE_METRIC) { Utils.bytesToString @@ -204,19 +204,21 @@ object SQLMetrics { } val validValues = values.filter(_ >= 0) - val Seq(sum, min, med, max) = { - val metric = if (validValues.isEmpty) { - val zeros = Seq.fill(4)(0L) - zeros.map(v => strFormat(v)) - } else { + // When there are only 1 metrics value (or None), no need to display max/min/median. This is + // common for driver-side SQL metrics. + if (validValues.length <= 1) { + strFormat(validValues.headOption.getOrElse(0)) + } else { + val Seq(sum, min, med, max) = { Arrays.sort(validValues) - Seq(strFormat(validValues.sum), strFormat(validValues(0)), + Seq( + strFormat(validValues.sum), + strFormat(validValues(0)), strFormat(validValues(validValues.length / 2)), - s"${strFormat(validValues(validValues.length - 1))} $stringMetric") + strFormat(validValues(validValues.length - 1))) } - metric + s"total $METRICS_NAME_SUFFIX\n$sum ($min, $med, $max $taskInfo)" } - s"\n$sum ($min, $med, $max)" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala index 6f1b67801bb80..bcd226f95f822 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala @@ -113,10 +113,8 @@ object FrequentItems extends Logging { val justItems = freqItems.map(m => m.baseMap.keys.toArray) val resultRow = Row(justItems : _*) - val originalSchema = df.schema val outputCols = cols.map { name => - val index = originalSchema.fieldIndex(name) - val originalField = originalSchema.fields(index) + val originalField = df.resolve(name) // append frequent Items to the column name for easy debugging StructField(name + "_freqItems", ArrayType(originalField.dataType, originalField.nullable)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index fffd8805a6525..5094e5eab5955 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -70,7 +70,7 @@ object StatFunctions extends Logging { require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError") val columns: Seq[Column] = cols.map { colName => - val field = df.schema(colName) + val field = df.resolve(colName) require(field.dataType.isInstanceOf[NumericType], s"Quantile calculation for column $colName with data type ${field.dataType}" + " is not supported.") @@ -154,10 +154,9 @@ object StatFunctions extends Logging { functionName: String): CovarianceCounter = { require(cols.length == 2, s"Currently $functionName calculation is supported " + "between two columns.") - cols.map(name => (name, df.schema.fields.find(_.name == name))).foreach { case (name, data) => - require(data.nonEmpty, s"Couldn't find column with name $name") - require(data.get.dataType.isInstanceOf[NumericType], s"Currently $functionName calculation " + - s"for columns with dataType ${data.get.dataType.catalogString} not supported.") + cols.map(name => (name, df.resolve(name))).foreach { case (name, data) => + require(data.dataType.isInstanceOf[NumericType], s"Currently $functionName calculation " + + s"for columns with dataType ${data.dataType.catalogString} not supported.") } val columns = cols.map(n => Column(Cast(Column(n).expr, DoubleType))) df.select(columns: _*).queryExecution.toRdd.treeAggregate(new CovarianceCounter)( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 45a2ce16183a5..e022bfb6835d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -226,7 +226,8 @@ class MicroBatchExecution( } } - finishTrigger(currentBatchHasNewData) // Must be outside reportTimeTaken so it is recorded + // Must be outside reportTimeTaken so it is recorded + finishTrigger(currentBatchHasNewData, isCurrentBatchConstructed) // Signal waiting threads. Note this must be after finishTrigger() to ensure all // activities (progress generation, etc.) have completed before signaling. 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 feb151a5468b8..0dff1c2fe5768 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 @@ -85,8 +85,8 @@ trait ProgressReporter extends Logging { private val noDataProgressEventInterval = sparkSession.sessionState.conf.streamingNoDataProgressEventInterval - // The timestamp we report an event that has no input data - private var lastNoDataProgressEventTime = Long.MinValue + // The timestamp we report an event that has not executed anything + private var lastNoExecutionProgressEventTime = Long.MinValue private val timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 timestampFormat.setTimeZone(DateTimeUtils.getTimeZone("UTC")) @@ -142,12 +142,19 @@ trait ProgressReporter extends Logging { logInfo(s"Streaming query made progress: $newProgress") } - /** Finalizes the query progress and adds it to list of recent status updates. */ - protected def finishTrigger(hasNewData: Boolean): Unit = { + /** + * Finalizes the query progress and adds it to list of recent status updates. + * + * @param hasNewData Whether the sources of this stream had new data for this trigger. + * @param hasExecuted Whether any batch was executed during this trigger. Streaming queries that + * perform stateful aggregations with timeouts can still run batches even + * though the sources don't have any new data. + */ + protected def finishTrigger(hasNewData: Boolean, hasExecuted: Boolean): Unit = { assert(currentTriggerStartOffsets != null && currentTriggerEndOffsets != null) currentTriggerEndTimestamp = triggerClock.getTimeMillis() - val executionStats = extractExecutionStats(hasNewData) + val executionStats = extractExecutionStats(hasNewData, hasExecuted) val processingTimeMills = currentTriggerEndTimestamp - currentTriggerStartTimestamp val processingTimeSec = Math.max(1L, processingTimeMills).toDouble / MILLIS_PER_SECOND @@ -170,9 +177,12 @@ trait ProgressReporter extends Logging { ) } - val sinkProgress = SinkProgress( - sink.toString, - sinkCommitProgress.map(_.numOutputRows)) + val sinkOutput = if (hasExecuted) { + sinkCommitProgress.map(_.numOutputRows) + } else { + sinkCommitProgress.map(_ => 0L) + } + val sinkProgress = SinkProgress(sink.toString, sinkOutput) val observedMetrics = extractObservedMetrics(hasNewData, lastExecution) val newProgress = new StreamingQueryProgress( @@ -189,14 +199,14 @@ trait ProgressReporter extends Logging { sink = sinkProgress, observedMetrics = new java.util.HashMap(observedMetrics.asJava)) - if (hasNewData) { + if (hasExecuted) { // Reset noDataEventTimestamp if we processed any data - lastNoDataProgressEventTime = Long.MinValue + lastNoExecutionProgressEventTime = Long.MinValue updateProgress(newProgress) } else { val now = triggerClock.getTimeMillis() - if (now - noDataProgressEventInterval >= lastNoDataProgressEventTime) { - lastNoDataProgressEventTime = now + if (now - noDataProgressEventInterval >= lastNoExecutionProgressEventTime) { + lastNoExecutionProgressEventTime = now updateProgress(newProgress) } } @@ -205,26 +215,26 @@ trait ProgressReporter extends Logging { } /** Extract statistics about stateful operators from the executed query plan. */ - private def extractStateOperatorMetrics(hasNewData: Boolean): Seq[StateOperatorProgress] = { + private def extractStateOperatorMetrics(hasExecuted: Boolean): Seq[StateOperatorProgress] = { if (lastExecution == null) return Nil - // lastExecution could belong to one of the previous triggers if `!hasNewData`. + // lastExecution could belong to one of the previous triggers if `!hasExecuted`. // Walking the plan again should be inexpensive. lastExecution.executedPlan.collect { case p if p.isInstanceOf[StateStoreWriter] => val progress = p.asInstanceOf[StateStoreWriter].getProgress() - if (hasNewData) progress else progress.copy(newNumRowsUpdated = 0) + if (hasExecuted) progress else progress.copy(newNumRowsUpdated = 0) } } /** Extracts statistics from the most recent query execution. */ - private def extractExecutionStats(hasNewData: Boolean): ExecutionStats = { + private def extractExecutionStats(hasNewData: Boolean, hasExecuted: Boolean): ExecutionStats = { val hasEventTime = logicalPlan.collect { case e: EventTimeWatermark => e }.nonEmpty val watermarkTimestamp = if (hasEventTime) Map("watermark" -> formatTimestamp(offsetSeqMetadata.batchWatermarkMs)) else Map.empty[String, String] // SPARK-19378: Still report metrics even though no data was processed while reporting progress. - val stateOperators = extractStateOperatorMetrics(hasNewData) + val stateOperators = extractStateOperatorMetrics(hasExecuted) if (!hasNewData) { return ExecutionStats(Map.empty, stateOperators, watermarkTimestamp) 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 8006437b31297..9b1951a834d9a 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 @@ -451,9 +451,9 @@ abstract class StreamExecution( val stackTraceException = new SparkException("The stream thread was last executing:") stackTraceException.setStackTrace(queryExecutionThread.getStackTrace) val timeoutException = new TimeoutException( - s"Stream Execution thread failed to stop within $timeout milliseconds (specified by " + - s"${SQLConf.STREAMING_STOP_TIMEOUT.key}). See the cause on what was " + - "being executed in the streaming query thread.") + s"Stream Execution thread for stream $prettyIdString failed to stop within $timeout " + + s"milliseconds (specified by ${SQLConf.STREAMING_STOP_TIMEOUT.key}). See the cause on " + + s"what was being executed in the streaming query thread.") timeoutException.initCause(stackTraceException) throw timeoutException } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala index 6084aaf6c5706..76bc7faf18d01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala @@ -71,6 +71,10 @@ class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging {jobLinks(JobExecutionStatus.FAILED, "Failed Jobs:")} +
    + + Show the Stage ID and Task ID that corresponds to the max metric +
    val metrics = sqlStore.executionMetrics(executionId) val graph = sqlStore.planGraph(executionId) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index d31d77840b802..274a5a414ffa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -160,22 +160,28 @@ private[ui] class SparkPlanGraphNode( val metrics: Seq[SQLPlanMetric]) { def makeDotNode(metricsValue: Map[Long, String]): String = { - val builder = new mutable.StringBuilder(name) + val builder = new mutable.StringBuilder("" + name + "") val values = for { metric <- metrics value <- metricsValue.get(metric.accumulatorId) } yield { - metric.name + ": " + value + // The value may contain ":" to extend the name, like `total (min, med, max): ...` + if (value.contains(":")) { + metric.name + " " + value + } else { + metric.name + ": " + value + } } if (values.nonEmpty) { // If there are metrics, display each entry in a separate line. // Note: whitespace between two "\n"s is to create an empty line between the name of // SparkPlan and metrics. If removing it, it won't display the empty line in UI. - builder ++= "\n \n" - builder ++= values.mkString("\n") - s""" $id [label="${StringEscapeUtils.escapeJava(builder.toString())}"];""" + builder ++= "

    " + builder ++= values.mkString("
    ") + val labelStr = StringEscapeUtils.escapeJava(builder.toString().replaceAll("\n", "
    ")) + s""" $id [labelType="html" label="${labelStr}"];""" } else { // SPARK-30684: when there is no metrics, add empty lines to increase the height of the node, // so that there won't be gaps between an edge and a small node. @@ -210,6 +216,7 @@ private[ui] class SparkPlanGraphCluster( } s""" | subgraph cluster${id} { + | isCluster="true"; | label="${StringEscapeUtils.escapeJava(labelStr)}"; | ${nodes.map(_.makeDotNode(metricsValue)).mkString(" \n")} | } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index c50168cf7ac13..2ef6e3d291cef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -93,7 +93,7 @@ sealed abstract class UserDefinedFunction { private[spark] case class SparkUserDefinedFunction( f: AnyRef, dataType: DataType, - inputSchemas: Seq[Option[ScalaReflection.Schema]], + inputEncoders: Seq[Option[ExpressionEncoder[_]]] = Nil, name: Option[String] = None, nullable: Boolean = true, deterministic: Boolean = true) extends UserDefinedFunction { @@ -104,18 +104,11 @@ private[spark] case class SparkUserDefinedFunction( } private[sql] def createScalaUDF(exprs: Seq[Expression]): ScalaUDF = { - // It's possible that some of the inputs don't have a specific type(e.g. `Any`), skip type - // check. - val inputTypes = inputSchemas.map(_.map(_.dataType).getOrElse(AnyDataType)) - // `ScalaReflection.Schema.nullable` is false iff the type is primitive. Also `Any` is not - // primitive. - val inputsPrimitive = inputSchemas.map(_.map(!_.nullable).getOrElse(false)) ScalaUDF( f, dataType, exprs, - inputsPrimitive, - inputTypes, + inputEncoders, udfName = name, nullable = nullable, udfDeterministic = deterministic) 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 e2d3d55812c51..653e1a739aaf1 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 @@ -211,6 +211,36 @@ object functions { // Aggregate functions ////////////////////////////////////////////////////////////////////////////////////////////// + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(e: Column): Column = approx_count_distinct(e) + + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(columnName: String): Column = approx_count_distinct(columnName) + + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(e: Column, rsd: Double): Column = approx_count_distinct(e, rsd) + + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(columnName: String, rsd: Double): Column = { + approx_count_distinct(Column(columnName), rsd) + } + /** * Aggregate function: returns the approximate number of distinct items in a group. * @@ -653,6 +683,29 @@ object functions { */ def min(columnName: String): Column = min(Column(columnName)) + /** + * Aggregate function: returns and array of the approximate percentile values + * of numeric column col at the given percentages. + * + * If percentage is an array, each value must be between 0.0 and 1.0. + * If it is a single floating point value, it must be between 0.0 and 1.0. + * + * The accuracy parameter is a positive numeric literal + * which controls approximation accuracy at the cost of memory. + * Higher value of accuracy yields better accuracy, 1.0/accuracy + * is the relative error of the approximation. + * + * @group agg_funcs + * @since 3.1.0 + */ + def percentile_approx(e: Column, percentage: Column, accuracy: Column): Column = { + withAggregateFunction { + new ApproximatePercentile( + e.expr, percentage.expr, accuracy.expr + ) + } + } + /** * Aggregate function: returns the skewness of the values in a group. * @@ -1090,6 +1143,27 @@ object functions { */ def isnull(e: Column): Column = withExpr { IsNull(e.expr) } + /** + * A column expression that generates monotonically increasing 64-bit integers. + * + * The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive. + * The current implementation puts the partition ID in the upper 31 bits, and the record number + * within each partition in the lower 33 bits. The assumption is that the data frame has + * less than 1 billion partitions, and each partition has less than 8 billion records. + * + * As an example, consider a `DataFrame` with two partitions, each with 3 records. + * This expression would return the following IDs: + * + * {{{ + * 0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594. + * }}} + * + * @group normal_funcs + * @since 1.4.0 + */ + @deprecated("Use monotonically_increasing_id()", "2.0.0") + def monotonicallyIncreasingId(): Column = monotonically_increasing_id() + /** * A column expression that generates monotonically increasing 64-bit integers. * @@ -1153,7 +1227,7 @@ object functions { /** * Generate a random column with independent and identically distributed (i.i.d.) samples - * from U[0.0, 1.0]. + * uniformly distributed in [0.0, 1.0). * * @note The function is non-deterministic in general case. * @@ -1164,7 +1238,7 @@ object functions { /** * Generate a random column with independent and identically distributed (i.i.d.) samples - * from U[0.0, 1.0]. + * uniformly distributed in [0.0, 1.0). * * @note The function is non-deterministic in general case. * @@ -2071,6 +2145,20 @@ object functions { */ def tanh(columnName: String): Column = tanh(Column(columnName)) + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use degrees", "2.1.0") + def toDegrees(e: Column): Column = degrees(e) + + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use degrees", "2.1.0") + def toDegrees(columnName: String): Column = degrees(Column(columnName)) + /** * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. * @@ -2093,6 +2181,20 @@ object functions { */ def degrees(columnName: String): Column = degrees(Column(columnName)) + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use radians", "2.1.0") + def toRadians(e: Column): Column = radians(e) + + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use radians", "2.1.0") + def toRadians(columnName: String): Column = radians(Column(columnName)) + /** * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. * @@ -2632,7 +2734,9 @@ object functions { * Converts a date/timestamp/string to a value of string in the format specified by the date * format given by the second argument. * - * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns + * See + * Datetime Patterns + * for valid date and time format patterns * * @param dateExpr A date, timestamp or string. If a string, the data must be in a format that * can be cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` @@ -2890,7 +2994,9 @@ object functions { * representing the timestamp of that moment in the current system time zone in the given * format. * - * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns + * See + * Datetime Patterns + * for valid date and time format patterns * * @param ut A number of a type that is castable to a long, such as string or integer. Can be * negative for timestamps before the unix epoch @@ -2934,7 +3040,9 @@ object functions { /** * Converts time string with given pattern to Unix timestamp (in seconds). * - * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns + * See + * Datetime Patterns + * for valid date and time format patterns * * @param s A date, timestamp or string. If a string, the data must be in a format that can be * cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` @@ -2962,7 +3070,9 @@ object functions { /** * Converts time string with the given pattern to timestamp. * - * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns + * See + * Datetime Patterns + * for valid date and time format patterns * * @param s A date, timestamp or string. If a string, the data must be in a format that can be * cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` @@ -2987,7 +3097,9 @@ object functions { /** * Converts the column into a `DateType` with a specified format * - * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns + * See + * Datetime Patterns + * for valid date and time format patterns * * @param e A date, timestamp or string. If a string, the data must be in a format that can be * cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` @@ -3047,8 +3159,12 @@ object functions { * * @param ts A date, timestamp or string. If a string, the data must be in a format that can be * cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param tz A string detailing the time zone that the input should be adjusted to, such as - * `Europe/London`, `PST` or `GMT+5` + * @param tz A string detailing the time zone ID that the input should be adjusted to. It should + * be in the format of either region-based zone IDs or zone offsets. Region IDs must + * have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in + * the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are + * supported as aliases of '+00:00'. Other short names are not recommended to use + * because they can be ambiguous. * @return A timestamp, or null if `ts` was a string that could not be cast to a timestamp or * `tz` was an invalid value * @group datetime_funcs @@ -3076,8 +3192,12 @@ object functions { * * @param ts A date, timestamp or string. If a string, the data must be in a format that can be * cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param tz A string detailing the time zone that the input belongs to, such as `Europe/London`, - * `PST` or `GMT+5` + * @param tz A string detailing the time zone ID that the input should be adjusted to. It should + * be in the format of either region-based zone IDs or zone offsets. Region IDs must + * have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in + * the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are + * supported as aliases of '+00:00'. Other short names are not recommended to use + * because they can be ambiguous. * @return A timestamp, or null if `ts` was a string that could not be cast to a timestamp or * `tz` was an invalid value * @group datetime_funcs @@ -3957,6 +4077,10 @@ object functions { /** * Returns length of array or map. * + * The function returns null for null input if spark.sql.legacy.sizeOfNull is set to false or + * spark.sql.ansi.enabled is set to true. Otherwise, the function returns -1 for null input. + * With the default settings, the function returns -1 for null input. + * * @group collection_funcs * @since 1.5.0 */ @@ -4268,7 +4392,7 @@ object functions { (0 to 10).foreach { x => val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) - val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"Try(ScalaReflection.schemaFor(typeTag[A$i])).toOption :: $s"}) + val inputEncoders = (1 to x).foldRight("Nil")((i, s) => {s"Try(ExpressionEncoder[A$i]()).toOption :: $s"}) println(s""" |/** | * Defines a Scala closure of $x arguments as user-defined function (UDF). @@ -4281,8 +4405,8 @@ object functions { | */ |def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - | val inputSchemas = $inputSchemas - | val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + | val inputEncoders = $inputEncoders + | val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) | if (nullable) udf else udf.asNonNullable() |}""".stripMargin) } @@ -4305,7 +4429,7 @@ object functions { | */ |def udf(f: UDF$i[$extTypeArgs], returnType: DataType): UserDefinedFunction = { | val func = $funcCall - | SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill($i)(None)) + | SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill($i)(None)) |}""".stripMargin) } @@ -4387,8 +4511,8 @@ object functions { */ def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4403,8 +4527,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4419,8 +4543,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4435,8 +4559,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4451,8 +4575,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4467,8 +4591,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4483,8 +4607,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4499,8 +4623,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4515,8 +4639,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A8])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4531,8 +4655,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A8])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A9])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4547,8 +4671,8 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A8])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A9])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A10])).toOption :: Nil - val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) + val inputEncoders = Try(ExpressionEncoder[A1]()).toOption :: Try(ExpressionEncoder[A2]()).toOption :: Try(ExpressionEncoder[A3]()).toOption :: Try(ExpressionEncoder[A4]()).toOption :: Try(ExpressionEncoder[A5]()).toOption :: Try(ExpressionEncoder[A6]()).toOption :: Try(ExpressionEncoder[A7]()).toOption :: Try(ExpressionEncoder[A8]()).toOption :: Try(ExpressionEncoder[A9]()).toOption :: Try(ExpressionEncoder[A10]()).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputEncoders) if (nullable) udf else udf.asNonNullable() } @@ -4567,7 +4691,7 @@ object functions { */ def udf(f: UDF0[_], returnType: DataType): UserDefinedFunction = { val func = () => f.asInstanceOf[UDF0[Any]].call() - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(0)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(0)(None)) } /** @@ -4581,7 +4705,7 @@ object functions { */ def udf(f: UDF1[_, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(1)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(1)(None)) } /** @@ -4595,7 +4719,7 @@ object functions { */ def udf(f: UDF2[_, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(2)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(2)(None)) } /** @@ -4609,7 +4733,7 @@ object functions { */ def udf(f: UDF3[_, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(3)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(3)(None)) } /** @@ -4623,7 +4747,7 @@ object functions { */ def udf(f: UDF4[_, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(4)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(4)(None)) } /** @@ -4637,7 +4761,7 @@ object functions { */ def udf(f: UDF5[_, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(5)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(5)(None)) } /** @@ -4651,7 +4775,7 @@ object functions { */ def udf(f: UDF6[_, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(6)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(6)(None)) } /** @@ -4665,7 +4789,7 @@ object functions { */ def udf(f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(7)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(7)(None)) } /** @@ -4679,7 +4803,7 @@ object functions { */ def udf(f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(8)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(8)(None)) } /** @@ -4693,7 +4817,7 @@ object functions { */ def udf(f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { 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) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(9)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(9)(None)) } /** @@ -4707,7 +4831,7 @@ object functions { */ def udf(f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { 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) - SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(10)(None)) + SparkUserDefinedFunction(func, returnType, inputEncoders = Seq.fill(10)(None)) } // scalastyle:on parameter.number @@ -4740,12 +4864,16 @@ object functions { "information. Spark may blindly pass null to the Scala closure with primitive-type " + "argument, and the closure will see the default value of the Java type for the null " + "argument, e.g. `udf((x: Int) => x, IntegerType)`, the result is 0 for null input. " + - "You could use typed Scala UDF APIs (e.g. `udf((x: Int) => x)`) to avoid this problem, " + - s"or set ${SQLConf.LEGACY_ALLOW_UNTYPED_SCALA_UDF.key} to true and use this API with " + - s"caution." + "To get rid of this error, you could:\n" + + "1. use typed Scala UDF APIs, e.g. `udf((x: Int) => x)`\n" + + "2. use Java UDF APIs, e.g. `udf(new UDF1[String, Integer] { " + + "override def call(s: String): Integer = s.length() }, IntegerType)`, " + + "if input types are all non primitive\n" + + s"3. set ${SQLConf.LEGACY_ALLOW_UNTYPED_SCALA_UDF.key} to true and " + + s"use this API with caution" throw new AnalysisException(errorMsg) } - SparkUserDefinedFunction(f, dataType, inputSchemas = Nil) + SparkUserDefinedFunction(f, dataType, inputEncoders = Nil) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 9556d4d1ecabf..3bbdbb002cca8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -347,8 +347,14 @@ private[sql] trait WithTestConf { self: BaseSessionStateBuilder => override protected lazy val conf: SQLConf = { val overrideConfigurations = overrideConfs - val conf = parentState.map(_.conf.clone()).getOrElse { - new SQLConf { + parentState.map { s => + val cloned = s.conf.clone() + if (session.sparkContext.conf.get(StaticSQLConf.SQL_LEGACY_SESSION_INIT_WITH_DEFAULTS)) { + mergeSparkConf(conf, session.sparkContext.conf) + } + cloned + }.getOrElse { + val conf = new SQLConf { clear() override def clear(): Unit = { super.clear() @@ -356,8 +362,8 @@ private[sql] trait WithTestConf { self: BaseSessionStateBuilder => overrideConfigurations.foreach { case (key, value) => setConfString(key, value) } } } + mergeSparkConf(conf, session.sparkContext.conf) + conf } - mergeSparkConf(conf, session.sparkContext.conf) - conf } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 5347264d7c50a..14b8ea66c8071 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -22,6 +22,7 @@ import java.util.UUID import java.util.concurrent.ConcurrentHashMap import javax.annotation.concurrent.GuardedBy +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.util.control.NonFatal @@ -41,7 +42,6 @@ import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, Streamin import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.Utils - /** * A class that holds all state shared across sessions in a given [[SQLContext]]. * @@ -55,45 +55,10 @@ private[sql] class SharedState( SharedState.setFsUrlStreamHandlerFactory(sparkContext.conf) - // Load hive-site.xml into hadoopConf and determine the warehouse path we want to use, based on - // the config from both hive and Spark SQL. Finally set the warehouse config value to sparkConf. - val warehousePath: String = { - val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml") - if (configFile != null) { - logInfo(s"loading hive config file: $configFile") - sparkContext.hadoopConfiguration.addResource(configFile) - } - - // hive.metastore.warehouse.dir only stay in hadoopConf - sparkContext.conf.remove("hive.metastore.warehouse.dir") - // Set the Hive metastore warehouse path to the one we use - val hiveWarehouseDir = sparkContext.hadoopConfiguration.get("hive.metastore.warehouse.dir") - if (hiveWarehouseDir != null && !sparkContext.conf.contains(WAREHOUSE_PATH.key)) { - // If hive.metastore.warehouse.dir is set and spark.sql.warehouse.dir is not set, - // we will respect the value of hive.metastore.warehouse.dir. - sparkContext.conf.set(WAREHOUSE_PATH.key, hiveWarehouseDir) - logInfo(s"${WAREHOUSE_PATH.key} is not set, but hive.metastore.warehouse.dir " + - s"is set. Setting ${WAREHOUSE_PATH.key} to the value of " + - s"hive.metastore.warehouse.dir ('$hiveWarehouseDir').") - hiveWarehouseDir - } else { - // If spark.sql.warehouse.dir is set, we will override hive.metastore.warehouse.dir using - // the value of spark.sql.warehouse.dir. - // When neither spark.sql.warehouse.dir nor hive.metastore.warehouse.dir is set, - // we will set hive.metastore.warehouse.dir to the default value of spark.sql.warehouse.dir. - val sparkWarehouseDir = sparkContext.conf.get(WAREHOUSE_PATH) - logInfo(s"Setting hive.metastore.warehouse.dir ('$hiveWarehouseDir') to the value of " + - s"${WAREHOUSE_PATH.key} ('$sparkWarehouseDir').") - sparkContext.hadoopConfiguration.set("hive.metastore.warehouse.dir", sparkWarehouseDir) - sparkWarehouseDir - } - } - logInfo(s"Warehouse path is '$warehousePath'.") - - // These 2 variables should be initiated after `warehousePath`, because in the first place we need - // to load hive-site.xml into hadoopConf and determine the warehouse path which will be set into - // both spark conf and hadoop conf avoiding be affected by any SparkSession level options private val (conf, hadoopConf) = { + // Load hive-site.xml into hadoopConf and determine the warehouse path which will be set into + // both spark conf and hadoop conf avoiding be affected by any SparkSession level options + SharedState.loadHiveConfFile(sparkContext.conf, sparkContext.hadoopConfiguration) val confClone = sparkContext.conf.clone() val hadoopConfClone = new Configuration(sparkContext.hadoopConfiguration) // If `SparkSession` is instantiated using an existing `SparkContext` instance and no existing @@ -166,7 +131,7 @@ private[sql] class SharedState( val defaultDbDefinition = CatalogDatabase( SessionCatalog.DEFAULT_DATABASE, "default database", - CatalogUtils.stringToURI(warehousePath), + CatalogUtils.stringToURI(conf.get(WAREHOUSE_PATH)), Map()) // Create default database if it doesn't exist if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) { @@ -258,4 +223,46 @@ object SharedState extends Logging { throw new IllegalArgumentException(s"Error while instantiating '$className':", e) } } + + /** + * Load hive-site.xml into hadoopConf and determine the warehouse path we want to use, based on + * the config from both hive and Spark SQL. Finally set the warehouse config value to sparkConf. + */ + def loadHiveConfFile( + sparkConf: SparkConf, + hadoopConf: Configuration): Unit = { + val hiveWarehouseKey = "hive.metastore.warehouse.dir" + val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml") + if (configFile != null) { + logInfo(s"loading hive config file: $configFile") + val hadoopConfTemp = new Configuration() + hadoopConfTemp.addResource(configFile) + hadoopConfTemp.asScala.foreach { entry => + hadoopConf.setIfUnset(entry.getKey, entry.getValue) + } + } + // hive.metastore.warehouse.dir only stay in hadoopConf + sparkConf.remove(hiveWarehouseKey) + // Set the Hive metastore warehouse path to the one we use + val hiveWarehouseDir = hadoopConf.get(hiveWarehouseKey) + val warehousePath = if (hiveWarehouseDir != null && !sparkConf.contains(WAREHOUSE_PATH.key)) { + // If hive.metastore.warehouse.dir is set and spark.sql.warehouse.dir is not set, + // we will respect the value of hive.metastore.warehouse.dir. + sparkConf.set(WAREHOUSE_PATH.key, hiveWarehouseDir) + logInfo(s"${WAREHOUSE_PATH.key} is not set, but $hiveWarehouseKey is set. Setting" + + s" ${WAREHOUSE_PATH.key} to the value of $hiveWarehouseKey ('$hiveWarehouseDir').") + hiveWarehouseDir + } else { + // If spark.sql.warehouse.dir is set, we will override hive.metastore.warehouse.dir using + // the value of spark.sql.warehouse.dir. + // When neither spark.sql.warehouse.dir nor hive.metastore.warehouse.dir is set + // we will set hive.metastore.warehouse.dir to the default value of spark.sql.warehouse.dir. + val sparkWarehouseDir = sparkConf.get(WAREHOUSE_PATH) + logInfo(s"Setting $hiveWarehouseKey ('$hiveWarehouseDir') to the value of " + + s"${WAREHOUSE_PATH.key} ('$sparkWarehouseDir').") + hadoopConf.set(hiveWarehouseKey, sparkWarehouseDir) + sparkWarehouseDir + } + logInfo(s"Warehouse path is '$warehousePath'.") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 61875931d226e..58de6758f4d7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -49,6 +49,7 @@ package object sql { * Metadata key which is used to write Spark version in the followings: * - Parquet file metadata * - ORC file metadata + * - Avro file metadata * * Note that Hive table property `spark.sql.create.version` also has Spark version. */ 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 6848be15d42a5..1d7e4d3059c68 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 @@ -81,8 +81,19 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to parse timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to parse timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 2.0.0 @@ -118,8 +129,19 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to parse timestamps in the JSON/CSV data sources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to parse timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 2.0.0 @@ -134,8 +156,19 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to parse timestamps in the JSON/CSV data sources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to parse timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 2.0.0 @@ -253,11 +286,15 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * Custom date formats follow the formats at + * + * Datetime Patterns. * This applies to date type.
  • - *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.time.format.DateTimeFormatter`. This applies to timestamp type.
  • + * + * Datetime Patterns. + * This applies to timestamp type. *
  • `multiLine` (default `false`): parse one record, which may span multiple lines, * per file
  • *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator @@ -319,11 +356,15 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
  • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity * value.
  • *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. - * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * Custom date formats follow the formats at + * + * Datetime Patterns. * This applies to date type.
  • - *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.time.format.DateTimeFormatter`. This applies to timestamp type.
  • + * + * Datetime Patterns. + * This applies to timestamp type. *
  • `maxColumns` (default `20480`): defines a hard limit of how many columns * a record can have.
  • *
  • `maxCharsPerColumn` (default `-1`): defines the maximum number of characters allowed 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 1c21a30dd5bd6..1d0ca4d9453a5 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 @@ -161,8 +161,19 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to format timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to format timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 2.0.0 @@ -198,8 +209,19 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to format timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to format timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 2.0.0 @@ -214,8 +236,19 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * You can set the following option(s): *
      - *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone - * to be used to format timestamps in the JSON/CSV datasources or partition values.
    • + *
    • `timeZone` (default session local timezone): sets the string that indicates a time zone ID + * to be used to format timestamps in the JSON/CSV datasources or partition values. The following + * formats of `timeZone` are supported: + *
        + *
      • Region-based zone ID: It should have the form 'area/city', such as + * 'America/Los_Angeles'.
      • + *
      • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' + * or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
      • + *
      + * Other short names like 'CST' are not recommended to use because they can be ambiguous. + * If it isn't set, the current value of the SQL config `spark.sql.session.timeZone` is + * used by default. + *
    • *
    * * @since 2.0.0 diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index b14778b91510e..fd3325085df96 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -58,20 +58,67 @@ select date_add('2011-11-11', 1L); select date_add('2011-11-11', 1.0); select date_add('2011-11-11', 1E1); select date_add('2011-11-11', '1'); +select date_add('2011-11-11', '1.2'); select date_add(date'2011-11-11', 1); select date_add(timestamp'2011-11-11', 1); select date_sub(date'2011-11-11', 1); +select date_sub(date'2011-11-11', '1'); +select date_sub(date'2011-11-11', '1.2'); select date_sub(timestamp'2011-11-11', 1); select date_sub(null, 1); select date_sub(date'2011-11-11', null); select date'2011-11-11' + 1E1; +select date'2011-11-11' + '1'; select null + date '2001-09-28'; select date '2001-09-28' + 7Y; select 7S + date '2001-09-28'; select date '2001-10-01' - 7; +select date '2001-10-01' - '7'; select date '2001-09-28' + null; select date '2001-09-28' - null; +-- date add/sub with non-literal string column +create temp view v as select '1' str; +select date_add('2011-11-11', str) from v; +select date_sub('2011-11-11', str) from v; + -- subtract dates select null - date '2019-10-06'; select date '2001-10-01' - date '2001-09-28'; + +-- variable-length tests +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +-- exceeded max variable length +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +-- special cases +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]'); +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]'); +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]'); +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]'); +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]'); +select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS"); +select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm"); +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm"); +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm"); +select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm"); + +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'"); +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd"); + +select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuee'); +select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uucc'); +select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuuu'); + +select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS"); -- middle +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''"); -- tail +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss"); -- head +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss"); -- head but as single quote diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql b/sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql index 821cb473751eb..d55e398329b76 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql @@ -52,7 +52,7 @@ DROP TABLE desc_complex_col_table; --Test case insensitive -CREATE TABLE customer(CName STRING); +CREATE TABLE customer(CName STRING) USING PARQUET; INSERT INTO customer VALUES('Maria'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/extract.sql b/sql/core/src/test/resources/sql-tests/inputs/extract.sql index 2d6cad8838704..4bcc15c875f57 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/extract.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/extract.sql @@ -1,43 +1,43 @@ -CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c; +CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c, interval 10 year 20 month 30 day 40 hour 50 minute 6.7890 second as i; -select extract(millennium from c) from t; -select extract(millennia from c) from t; -select extract(mil from c) from t; -select extract(mils from c) from t; +select extract(millennium from c), extract(millennium from i) from t; +select extract(millennia from c), extract(millennia from i) from t; +select extract(mil from c), extract(mil from i) from t; +select extract(mils from c), extract(mils from i) from t; -select extract(century from c) from t; -select extract(centuries from c) from t; -select extract(c from c) from t; -select extract(cent from c) from t; +select extract(century from c), extract(century from i) from t; +select extract(centuries from c), extract(centuries from i) from t; +select extract(c from c), extract(c from i) from t; +select extract(cent from c), extract(cent from i) from t; -select extract(decade from c) from t; -select extract(decades from c) from t; -select extract(dec from c) from t; -select extract(decs from c) from t; +select extract(decade from c), extract(decade from i) from t; +select extract(decades from c), extract(decades from i) from t; +select extract(dec from c), extract(dec from i) from t; +select extract(decs from c), extract(decs from i) from t; -select extract(year from c) from t; -select extract(y from c) from t; -select extract(years from c) from t; -select extract(yr from c) from t; -select extract(yrs from c) from t; +select extract(year from c), extract(year from i) from t; +select extract(y from c), extract(y from i) from t; +select extract(years from c), extract(years from i) from t; +select extract(yr from c), extract(yr from i) from t; +select extract(yrs from c), extract(yrs from i) from t; select extract(isoyear from c) from t; -select extract(quarter from c) from t; -select extract(qtr from c) from t; +select extract(quarter from c), extract(quarter from i) from t; +select extract(qtr from c), extract(qtr from i) from t; -select extract(month from c) from t; -select extract(mon from c) from t; -select extract(mons from c) from t; -select extract(months from c) from t; +select extract(month from c), extract(month from i) from t; +select extract(mon from c), extract(mon from i) from t; +select extract(mons from c), extract(mons from i) from t; +select extract(months from c), extract(months from i) from t; select extract(week from c) from t; select extract(w from c) from t; select extract(weeks from c) from t; -select extract(day from c) from t; -select extract(d from c) from t; -select extract(days from c) from t; +select extract(day from c), extract(day from i) from t; +select extract(d from c), extract(d from i) from t; +select extract(days from c), extract(days from i) from t; select extract(dayofweek from c) from t; @@ -47,38 +47,39 @@ select extract(isodow from c) from t; select extract(doy from c) from t; -select extract(hour from c) from t; -select extract(h from c) from t; -select extract(hours from c) from t; -select extract(hr from c) from t; -select extract(hrs from c) from t; - -select extract(minute from c) from t; -select extract(m from c) from t; -select extract(min from c) from t; -select extract(mins from c) from t; -select extract(minutes from c) from t; - -select extract(second from c) from t; -select extract(s from c) from t; -select extract(sec from c) from t; -select extract(seconds from c) from t; -select extract(secs from c) from t; - -select extract(milliseconds from c) from t; -select extract(msec from c) from t; -select extract(msecs from c) from t; -select extract(millisecon from c) from t; -select extract(mseconds from c) from t; -select extract(ms from c) from t; - -select extract(microseconds from c) from t; -select extract(usec from c) from t; -select extract(usecs from c) from t; -select extract(useconds from c) from t; -select extract(microsecon from c) from t; -select extract(us from c) from t; - -select extract(epoch from c) from t; +select extract(hour from c), extract(hour from i) from t; +select extract(h from c), extract(h from i) from t; +select extract(hours from c), extract(hours from i) from t; +select extract(hr from c), extract(hr from i) from t; +select extract(hrs from c), extract(hrs from i) from t; + +select extract(minute from c), extract(minute from i) from t; +select extract(m from c), extract(m from i) from t; +select extract(min from c), extract(min from i) from t; +select extract(mins from c), extract(mins from i) from t; +select extract(minutes from c), extract(minutes from i) from t; + +select extract(second from c), extract(second from i) from t; +select extract(s from c), extract(s from i) from t; +select extract(sec from c), extract(sec from i) from t; +select extract(seconds from c), extract(seconds from i) from t; +select extract(secs from c), extract(secs from i) from t; + +select extract(milliseconds from c), extract(milliseconds from i) from t; +select extract(msec from c), extract(msec from i) from t; +select extract(msecs from c), extract(msecs from i) from t; +select extract(millisecon from c), extract(millisecon from i) from t; +select extract(mseconds from c), extract(mseconds from i) from t; +select extract(ms from c), extract(ms from i) from t; + +select extract(microseconds from c), extract(microseconds from i) from t; +select extract(usec from c), extract(usec from i) from t; +select extract(usecs from c), extract(usecs from i) from t; +select extract(useconds from c), extract(useconds from i) from t; +select extract(microsecon from c), extract(microsecon from i) from t; +select extract(us from c), extract(us from i) from t; + +select extract(epoch from c), extract(epoch from i) from t; select extract(not_supported from c) from t; +select extract(not_supported from i) from t; diff --git a/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql index 6c14eee2e4e61..131890fddb0db 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql @@ -58,5 +58,35 @@ select schema_of_json('{"c1":01, "c2":0.1}', map('allowNumericLeadingZeros', 'tr select schema_of_json(null); CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a'); SELECT schema_of_json(jsonField) FROM jsonTable; + +-- json_array_length +select json_array_length(null); +select json_array_length(2); +select json_array_length(); +select json_array_length(''); +select json_array_length('[]'); +select json_array_length('[1,2,3]'); +select json_array_length('[[1,2],[5,6,7]]'); +select json_array_length('[{"a":123},{"b":"hello"}]'); +select json_array_length('[1,2,3,[33,44],{"key":[2,3,4]}]'); +select json_array_length('{"key":"not a json array"}'); +select json_array_length('[1,2,3,4,5'); + +-- json_object_keys +select json_object_keys(); +select json_object_keys(null); +select json_object_keys(200); +select json_object_keys(''); +select json_object_keys('{}'); +select json_object_keys('{"key": 1}'); +select json_object_keys('{"key": "value", "key2": 2}'); +select json_object_keys('{"arrayKey": [1, 2, 3]}'); +select json_object_keys('{"key":[1,2,3,{"key":"value"},[1,2,3]]}'); +select json_object_keys('{"f1":"abc","f2":{"f3":"a", "f4":"b"}}'); +select json_object_keys('{"k1": [1, 2, {"key": 5}], "k2": {"key2": [1, 2]}}'); +select json_object_keys('{[1,2]}'); +select json_object_keys('{"key": 45, "random_string"}'); +select json_object_keys('[1, 2, 3]'); + -- Clean up DROP VIEW IF EXISTS jsonTable; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/create_view.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/create_view.sql index 39e708478e298..21ffd85f7d01f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/create_view.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/create_view.sql @@ -41,7 +41,7 @@ DROP TABLE emp; -- These views are left around mainly to exercise special cases in pg_dump. -- [SPARK-19842] Informational Referential Integrity Constraints Support in Spark -CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)); +CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) USING PARQUET; -- CREATE VIEW key_dependent_view AS SELECT * FROM view_base_table GROUP BY key; diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql b/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql index dc77f87d9743a..00b46d1951fcf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql @@ -74,6 +74,9 @@ CREATE VIEW view_SPARK_30302 (aaa, bbb) AS SELECT a, b FROM tbl; SHOW CREATE TABLE view_SPARK_30302 AS SERDE; + +SHOW CREATE TABLE view_SPARK_30302; + DROP VIEW view_SPARK_30302; @@ -83,6 +86,9 @@ COMMENT 'This is a comment with \'quoted text\' for view' AS SELECT a, b FROM tbl; SHOW CREATE TABLE view_SPARK_30302 AS SERDE; + +SHOW CREATE TABLE view_SPARK_30302; + DROP VIEW view_SPARK_30302; @@ -92,13 +98,9 @@ TBLPROPERTIES ('a' = '1', 'b' = '2') AS SELECT a, b FROM tbl; SHOW CREATE TABLE view_SPARK_30302 AS SERDE; -DROP VIEW view_SPARK_30302; - --- SHOW CREATE TABLE does not support view -CREATE VIEW view_SPARK_30302 (aaa, bbb) -AS SELECT a, b FROM tbl; SHOW CREATE TABLE view_SPARK_30302; + DROP VIEW view_SPARK_30302; DROP TABLE tbl; 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 3c77c9977d80f..8f46c93ee3233 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 @@ -15,6 +15,7 @@ SHOW TABLES IN showdb; SHOW TABLES 'show_t*'; SHOW TABLES LIKE 'show_t1*|show_t2*'; SHOW TABLES IN showdb 'show_t*'; +SHOW TABLES IN showdb LIKE 'show_t*'; -- SHOW TABLE EXTENDED SHOW TABLE EXTENDED LIKE 'show_t*'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-views.sql b/sql/core/src/test/resources/sql-tests/inputs/show-views.sql new file mode 100644 index 0000000000000..bdf9ef4aa6780 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/show-views.sql @@ -0,0 +1,28 @@ +-- Test data. +CREATE DATABASE showdb; +USE showdb; +CREATE TABLE tbl(a STRING, b INT, c STRING, d STRING) USING parquet; +CREATE VIEW view_1 AS SELECT * FROM tbl; +CREATE VIEW view_2 AS SELECT * FROM tbl WHERE c='a'; +CREATE GLOBAL TEMP VIEW view_3 AS SELECT 1 as col1; +CREATE TEMPORARY VIEW view_4(e INT) USING parquet; + +-- SHOW VIEWS +SHOW VIEWS; +SHOW VIEWS FROM showdb; +SHOW VIEWS IN showdb; +SHOW VIEWS IN global_temp; + +-- SHOW VIEWS WITH wildcard match +SHOW VIEWS 'view_*'; +SHOW VIEWS LIKE 'view_1*|view_2*'; +SHOW VIEWS IN showdb 'view_*'; +SHOW VIEWS IN showdb LIKE 'view_*'; +-- Error when database not exists +SHOW VIEWS IN wrongdb LIKE 'view_*'; + +-- Clean Up +DROP VIEW global_temp.view_3; +DROP VIEW view_4; +USE default; +DROP DATABASE showdb CASCADE; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql new file mode 100644 index 0000000000000..2f6835b59fdd5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql @@ -0,0 +1,198 @@ +-- Tests NOT-IN subqueries nested inside OR expression(s). + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", 10), + (200, "emp 2", NULL), + (300, "emp 3", 20), + (400, "emp 4", 30), + (500, "emp 5", NULL), + (600, "emp 6", 100), + (800, "emp 8", 70) +AS EMP(id, emp_name, 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); + +CREATE TEMPORARY VIEW ADDRESS AS SELECT * FROM VALUES + (100, "emp 1", "addr1"), + (200, null, "addr2"), + (null, "emp 3", "addr3"), + (null, null, "addr4"), + (600, "emp 6", "addr6"), + (800, "emp 8", "addr8") +AS ADDRESS(id, emp_name, address); + +CREATE TEMPORARY VIEW S1 AS SELECT * FROM VALUES + (null, null), (5, 5), (8, 8), (11, 11) AS s1(a, b); +CREATE TEMPORARY VIEW S2 AS SELECT * FROM VALUES + (7, 7), (8, 8), (11, 11), (null, null) AS s2(c, d); + +-- null produced from both sides. +-- TC.01.01 +SELECT id, + dept_id +FROM emp +WHERE id = 600 + OR id = 500 + OR dept_id NOT IN (SELECT dept_id + FROM emp); + +-- null produced from right side +-- TC.01.02 +SELECT id, + dept_id +FROM emp +WHERE id = 800 + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM emp)); + +-- null produced on left side +-- TC.01.03 +SELECT id, + dept_id +FROM emp +WHERE id = 100 + OR dept_id NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL); + +-- no null in both left and right +-- TC.01.04 +SELECT id, + dept_id +FROM emp +WHERE id = 200 + OR (dept_id IS NOT NULL + AND dept_id + 100 NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL)); + +-- complex nesting +-- TC.01.05 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE emp_name IN (SELECT emp_name + FROM bonus) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)); + +-- complex nesting, exists in disjunction with not-in +-- TC.01.06 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE EXISTS (SELECT emp_name + FROM bonus + WHERE emp.emp_name = bonus.emp_name) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)); + +-- multiple columns in not-in +-- TC.01.07 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 +OR (id, emp_name) NOT IN (SELECT id, emp_name FROM address); + +-- multiple columns in not-in +-- TC.01.08 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR (( id, emp_name ) NOT IN (SELECT id, + emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL) + AND id > 400 ); +-- correlated not-in along with disjunction +-- TC.01.09 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id); + +-- multiple not-in(s) in side disjunction` +-- TC.01.10 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE id NOT IN (SELECT id + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND id >= 400) + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id + AND id < 400); + +-- NOT (NOT IN (SUBQ)) +SELECT * +FROM s1 +WHERE NOT (a NOT IN (SELECT c + FROM s2)); + +-- NOT (OR (expression, IN-SUBQ)) +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a IN (SELECT c + FROM s2)); + +-- NOT (OR (expression, NOT-IN-SUB) +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a NOT IN (SELECT c + FROM s2)); + +-- NOT (AND (expression, IN-SUB)) +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a IN (SELECT c + FROM s2)); + +-- NOT (AND (expression, NOT-IN-SUBQ)) +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a NOT IN (SELECT c + FROM s2)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-join.sql index e6fe1078b0d24..77bcfe7e10cbc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-join.sql @@ -8,6 +8,9 @@ -- -- This test file was converted from postgreSQL/join.sql. +-- Disable BroadcastHashJoin optimization to avoid changing result order when we enable AQE +--SET spark.sql.autoBroadcastJoinThreshold = -1 + CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) AS v(f1); diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index a7b098d79a706..aec04da4dd54d 100755 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 47 +-- Number of queries: 85 -- !query @@ -266,10 +266,18 @@ cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mi -- !query select date_add('2011-11-11', '1') -- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', '1.2') +-- !query schema struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'date_add(CAST('2011-11-11' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 +The second argument of 'date_add' function needs to be an integer.; -- !query @@ -296,6 +304,23 @@ struct 2011-11-10 +-- !query +select date_sub(date'2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(date'2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +The second argument of 'date_sub' function needs to be an integer.; + + -- !query select date_sub(timestamp'2011-11-11', 1) -- !query schema @@ -329,6 +354,15 @@ org.apache.spark.sql.AnalysisException cannot resolve 'date_add(DATE '2011-11-11', 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7 +-- !query +select date'2011-11-11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(DATE '2011-11-11', CAST('1' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DOUBLE)' is of double type.; line 1 pos 7 + + -- !query select null + date '2001-09-28' -- !query schema @@ -361,6 +395,15 @@ struct 2001-09-24 +-- !query +select date '2001-10-01' - '7' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_sub(DATE '2001-10-01', CAST('7' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('7' AS DOUBLE)' is of double type.; line 1 pos 7 + + -- !query select date '2001-09-28' + null -- !query schema @@ -377,6 +420,32 @@ struct NULL +-- !query +create temp view v as select '1' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select date_add('2011-11-11', str) from v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), v.`str`)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'v.`str`' is of string type.; line 1 pos 7 + + +-- !query +select date_sub('2011-11-11', str) from v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), v.`str`)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'v.`str`' is of string type.; line 1 pos 7 + + -- !query select null - date '2019-10-06' -- !query schema @@ -391,3 +460,245 @@ select date '2001-10-01' - date '2001-09-28' struct -- !query output 3 days + + +-- !query +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 03:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 08:11:12.12345 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:00 + + +-- !query +select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12345 + + +-- !query +select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +0019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuee') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: e + + +-- !query +select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uucc') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character: c + + +-- !query +select date_format(timestamp '2019-10-06', 'yyyy-MM-dd uuuu') +-- !query schema +struct +-- !query output +2019-10-06 Sunday + + +-- !query +select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 diff --git a/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out index 6b16aba268f50..2199fc0312d25 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out @@ -112,7 +112,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input 'desc_temp1' expecting {, '.'}(line 1, pos 21) +mismatched input 'desc_temp1' expecting {, ';'}(line 1, pos 21) == SQL == DESCRIBE INSERT INTO desc_temp1 values (1, 'val1') @@ -126,7 +126,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input 'desc_temp1' expecting {, '.'}(line 1, pos 21) +mismatched input 'desc_temp1' expecting {, ';'}(line 1, pos 21) == SQL == DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2 diff --git a/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out index ae9240ec588da..c6d3d45879eb1 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out @@ -267,7 +267,7 @@ struct<> -- !query -CREATE TABLE customer(CName STRING) +CREATE TABLE customer(CName STRING) USING PARQUET -- !query schema struct<> -- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/extract.sql.out b/sql/core/src/test/resources/sql-tests/results/extract.sql.out index 583459f9037b8..38c8490694088 100644 --- a/sql/core/src/test/resources/sql-tests/results/extract.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/extract.sql.out @@ -1,9 +1,9 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 64 +-- Number of queries: 65 -- !query -CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c +CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c, interval 10 year 20 month 30 day 40 hour 50 minute 6.7890 second as i -- !query schema struct<> -- !query output @@ -11,201 +11,201 @@ struct<> -- !query -select extract(millennium from c) from t +select extract(millennium from c), extract(millennium from i) from t -- !query schema -struct +struct -- !query output -3 +3 0 -- !query -select extract(millennia from c) from t +select extract(millennia from c), extract(millennia from i) from t -- !query schema -struct +struct -- !query output -3 +3 0 -- !query -select extract(mil from c) from t +select extract(mil from c), extract(mil from i) from t -- !query schema -struct +struct -- !query output -3 +3 0 -- !query -select extract(mils from c) from t +select extract(mils from c), extract(mils from i) from t -- !query schema -struct +struct -- !query output -3 +3 0 -- !query -select extract(century from c) from t +select extract(century from c), extract(century from i) from t -- !query schema -struct +struct -- !query output -21 +21 0 -- !query -select extract(centuries from c) from t +select extract(centuries from c), extract(centuries from i) from t -- !query schema -struct +struct -- !query output -21 +21 0 -- !query -select extract(c from c) from t +select extract(c from c), extract(c from i) from t -- !query schema -struct +struct -- !query output -21 +21 0 -- !query -select extract(cent from c) from t +select extract(cent from c), extract(cent from i) from t -- !query schema -struct +struct -- !query output -21 +21 0 -- !query -select extract(decade from c) from t +select extract(decade from c), extract(decade from i) from t -- !query schema -struct +struct -- !query output -201 +201 1 -- !query -select extract(decades from c) from t +select extract(decades from c), extract(decades from i) from t -- !query schema -struct +struct -- !query output -201 +201 1 -- !query -select extract(dec from c) from t +select extract(dec from c), extract(dec from i) from t -- !query schema -struct +struct -- !query output -201 +201 1 -- !query -select extract(decs from c) from t +select extract(decs from c), extract(decs from i) from t -- !query schema -struct +struct -- !query output -201 +201 1 -- !query -select extract(year from c) from t +select extract(year from c), extract(year from i) from t -- !query schema -struct +struct -- !query output -2011 +2011 11 -- !query -select extract(y from c) from t +select extract(y from c), extract(y from i) from t -- !query schema -struct +struct -- !query output -2011 +2011 11 -- !query -select extract(years from c) from t +select extract(years from c), extract(years from i) from t -- !query schema -struct +struct -- !query output -2011 +2011 11 -- !query -select extract(yr from c) from t +select extract(yr from c), extract(yr from i) from t -- !query schema -struct +struct -- !query output -2011 +2011 11 -- !query -select extract(yrs from c) from t +select extract(yrs from c), extract(yrs from i) from t -- !query schema -struct +struct -- !query output -2011 +2011 11 -- !query select extract(isoyear from c) from t -- !query schema -struct +struct -- !query output 2011 -- !query -select extract(quarter from c) from t +select extract(quarter from c), extract(quarter from i) from t -- !query schema -struct +struct -- !query output -2 +2 3 -- !query -select extract(qtr from c) from t +select extract(qtr from c), extract(qtr from i) from t -- !query schema -struct +struct -- !query output -2 +2 3 -- !query -select extract(month from c) from t +select extract(month from c), extract(month from i) from t -- !query schema -struct +struct -- !query output -5 +5 8 -- !query -select extract(mon from c) from t +select extract(mon from c), extract(mon from i) from t -- !query schema -struct +struct -- !query output -5 +5 8 -- !query -select extract(mons from c) from t +select extract(mons from c), extract(mons from i) from t -- !query schema -struct +struct -- !query output -5 +5 8 -- !query -select extract(months from c) from t +select extract(months from c), extract(months from i) from t -- !query schema -struct +struct -- !query output -5 +5 8 -- !query select extract(week from c) from t -- !query schema -struct +struct -- !query output 18 @@ -213,7 +213,7 @@ struct -- !query select extract(w from c) from t -- !query schema -struct +struct -- !query output 18 @@ -221,39 +221,39 @@ struct -- !query select extract(weeks from c) from t -- !query schema -struct +struct -- !query output 18 -- !query -select extract(day from c) from t +select extract(day from c), extract(day from i) from t -- !query schema -struct +struct -- !query output -6 +6 30 -- !query -select extract(d from c) from t +select extract(d from c), extract(d from i) from t -- !query schema -struct +struct -- !query output -6 +6 30 -- !query -select extract(days from c) from t +select extract(days from c), extract(days from i) from t -- !query schema -struct +struct -- !query output -6 +6 30 -- !query select extract(dayofweek from c) from t -- !query schema -struct +struct -- !query output 6 @@ -261,7 +261,7 @@ struct -- !query select extract(dow from c) from t -- !query schema -struct +struct -- !query output 5 @@ -269,7 +269,7 @@ struct -- !query select extract(isodow from c) from t -- !query schema -struct +struct -- !query output 5 @@ -277,233 +277,233 @@ struct -- !query select extract(doy from c) from t -- !query schema -struct +struct -- !query output 126 -- !query -select extract(hour from c) from t +select extract(hour from c), extract(hour from i) from t -- !query schema -struct +struct -- !query output -7 +7 40 -- !query -select extract(h from c) from t +select extract(h from c), extract(h from i) from t -- !query schema -struct +struct -- !query output -7 +7 40 -- !query -select extract(hours from c) from t +select extract(hours from c), extract(hours from i) from t -- !query schema -struct +struct -- !query output -7 +7 40 -- !query -select extract(hr from c) from t +select extract(hr from c), extract(hr from i) from t -- !query schema -struct +struct -- !query output -7 +7 40 -- !query -select extract(hrs from c) from t +select extract(hrs from c), extract(hrs from i) from t -- !query schema -struct +struct -- !query output -7 +7 40 -- !query -select extract(minute from c) from t +select extract(minute from c), extract(minute from i) from t -- !query schema -struct +struct -- !query output -8 +8 50 -- !query -select extract(m from c) from t +select extract(m from c), extract(m from i) from t -- !query schema -struct +struct -- !query output -8 +8 50 -- !query -select extract(min from c) from t +select extract(min from c), extract(min from i) from t -- !query schema -struct +struct -- !query output -8 +8 50 -- !query -select extract(mins from c) from t +select extract(mins from c), extract(mins from i) from t -- !query schema -struct +struct -- !query output -8 +8 50 -- !query -select extract(minutes from c) from t +select extract(minutes from c), extract(minutes from i) from t -- !query schema -struct +struct -- !query output -8 +8 50 -- !query -select extract(second from c) from t +select extract(second from c), extract(second from i) from t -- !query schema -struct +struct -- !query output -9.123456 +9.123456 6.789000 -- !query -select extract(s from c) from t +select extract(s from c), extract(s from i) from t -- !query schema -struct +struct -- !query output -9.123456 +9.123456 6.789000 -- !query -select extract(sec from c) from t +select extract(sec from c), extract(sec from i) from t -- !query schema -struct +struct -- !query output -9.123456 +9.123456 6.789000 -- !query -select extract(seconds from c) from t +select extract(seconds from c), extract(seconds from i) from t -- !query schema -struct +struct -- !query output -9.123456 +9.123456 6.789000 -- !query -select extract(secs from c) from t +select extract(secs from c), extract(secs from i) from t -- !query schema -struct +struct -- !query output -9.123456 +9.123456 6.789000 -- !query -select extract(milliseconds from c) from t +select extract(milliseconds from c), extract(milliseconds from i) from t -- !query schema -struct +struct -- !query output -9123.456 +9123.456 6789.000 -- !query -select extract(msec from c) from t +select extract(msec from c), extract(msec from i) from t -- !query schema -struct +struct -- !query output -9123.456 +9123.456 6789.000 -- !query -select extract(msecs from c) from t +select extract(msecs from c), extract(msecs from i) from t -- !query schema -struct +struct -- !query output -9123.456 +9123.456 6789.000 -- !query -select extract(millisecon from c) from t +select extract(millisecon from c), extract(millisecon from i) from t -- !query schema -struct +struct -- !query output -9123.456 +9123.456 6789.000 -- !query -select extract(mseconds from c) from t +select extract(mseconds from c), extract(mseconds from i) from t -- !query schema -struct +struct -- !query output -9123.456 +9123.456 6789.000 -- !query -select extract(ms from c) from t +select extract(ms from c), extract(ms from i) from t -- !query schema -struct +struct -- !query output -9123.456 +9123.456 6789.000 -- !query -select extract(microseconds from c) from t +select extract(microseconds from c), extract(microseconds from i) from t -- !query schema -struct +struct -- !query output -9123456 +9123456 6789000 -- !query -select extract(usec from c) from t +select extract(usec from c), extract(usec from i) from t -- !query schema -struct +struct -- !query output -9123456 +9123456 6789000 -- !query -select extract(usecs from c) from t +select extract(usecs from c), extract(usecs from i) from t -- !query schema -struct +struct -- !query output -9123456 +9123456 6789000 -- !query -select extract(useconds from c) from t +select extract(useconds from c), extract(useconds from i) from t -- !query schema -struct +struct -- !query output -9123456 +9123456 6789000 -- !query -select extract(microsecon from c) from t +select extract(microsecon from c), extract(microsecon from i) from t -- !query schema -struct +struct -- !query output -9123456 +9123456 6789000 -- !query -select extract(us from c) from t +select extract(us from c), extract(us from i) from t -- !query schema -struct +struct -- !query output -9123456 +9123456 6789000 -- !query -select extract(epoch from c) from t +select extract(epoch from c), extract(epoch from i) from t -- !query schema -struct +struct -- !query output -1304665689.123456 +1304665689.123456 370608606.789000 -- !query @@ -511,10 +511,14 @@ select extract(not_supported from c) from t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.AnalysisException +Literals of type 'not_supported' are currently not supported for the string type.;; line 1 pos 7 -Literals of type 'not_supported' are currently not supported.(line 1, pos 7) -== SQL == -select extract(not_supported from c) from t --------^^^ +-- !query +select extract(not_supported from i) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Literals of type 'not_supported' are currently not supported for the interval type.;; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out index 401fbe0e4591a..7089e10cdef27 100644 --- a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out @@ -138,7 +138,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -extraneous input 'ROLLUP' expecting (line 1, pos 53) +extraneous input 'ROLLUP' expecting {, ';'}(line 1, pos 53) == SQL == SELECT a, b, c, count(d) FROM grouping GROUP BY WITH ROLLUP @@ -152,7 +152,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -extraneous input 'CUBE' expecting (line 1, pos 53) +extraneous input 'CUBE' expecting {, ';'}(line 1, pos 53) == SQL == SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 7a3dd74b33bb5..0d758ca1f24ca 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -639,7 +639,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -extraneous input 'day' expecting (line 1, pos 27) +extraneous input 'day' expecting {, ';'}(line 1, pos 27) == SQL == select interval 30 day day day diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index 920b45a8fa77c..866fd1245d0ed 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 42 +-- Number of queries: 67 -- !query @@ -346,6 +346,210 @@ org.apache.spark.sql.AnalysisException cannot resolve 'schema_of_json(jsontable.`jsonField`)' due to data type mismatch: The input json should be a foldable string expression and not null; however, got jsontable.`jsonField`.; line 1 pos 7 +-- !query +select json_array_length(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_array_length(2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'json_array_length(2)' due to data type mismatch: argument 1 requires string type, however, '2' is of int type.; line 1 pos 7 + + +-- !query +select json_array_length() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7 + + +-- !query +select json_array_length('') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_array_length('[]') +-- !query schema +struct +-- !query output +0 + + +-- !query +select json_array_length('[1,2,3]') +-- !query schema +struct +-- !query output +3 + + +-- !query +select json_array_length('[[1,2],[5,6,7]]') +-- !query schema +struct +-- !query output +2 + + +-- !query +select json_array_length('[{"a":123},{"b":"hello"}]') +-- !query schema +struct +-- !query output +2 + + +-- !query +select json_array_length('[1,2,3,[33,44],{"key":[2,3,4]}]') +-- !query schema +struct +-- !query output +5 + + +-- !query +select json_array_length('{"key":"not a json array"}') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_array_length('[1,2,3,4,5') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_object_keys() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function json_object_keys. Expected: 1; Found: 0; line 1 pos 7 + + +-- !query +select json_object_keys(null) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select json_object_keys(200) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +cannot resolve 'json_object_keys(200)' due to data type mismatch: argument 1 requires string type, however, '200' is of int type.; line 1 pos 7 + + +-- !query +select json_object_keys('') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select json_object_keys('{}') +-- !query schema +struct> +-- !query output +[] + + +-- !query +select json_object_keys('{"key": 1}') +-- !query schema +struct> +-- !query output +["key"] + + +-- !query +select json_object_keys('{"key": "value", "key2": 2}') +-- !query schema +struct> +-- !query output +["key","key2"] + + +-- !query +select json_object_keys('{"arrayKey": [1, 2, 3]}') +-- !query schema +struct> +-- !query output +["arrayKey"] + + +-- !query +select json_object_keys('{"key":[1,2,3,{"key":"value"},[1,2,3]]}') +-- !query schema +struct> +-- !query output +["key"] + + +-- !query +select json_object_keys('{"f1":"abc","f2":{"f3":"a", "f4":"b"}}') +-- !query schema +struct> +-- !query output +["f1","f2"] + + +-- !query +select json_object_keys('{"k1": [1, 2, {"key": 5}], "k2": {"key2": [1, 2]}}') +-- !query schema +struct> +-- !query output +["k1","k2"] + + +-- !query +select json_object_keys('{[1,2]}') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select json_object_keys('{"key": 45, "random_string"}') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select json_object_keys('[1, 2, 3]') +-- !query schema +struct> +-- !query output +NULL + + -- !query DROP VIEW IF EXISTS jsonTable -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/limit.sql.out index 281326e22a97a..074e7a6d28c47 100644 --- a/sql/core/src/test/resources/sql-tests/results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/limit.sql.out @@ -88,7 +88,7 @@ SELECT * FROM testdata LIMIT key > 3 struct<> -- !query output org.apache.spark.sql.AnalysisException -The limit expression must evaluate to a constant value, but got (testdata.`key` > 3); +The limit expression must evaluate to a constant value, but got (spark_catalog.default.testdata.`key` > 3); -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index 083410f8807c4..cf857cf9f98ad 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -293,7 +293,7 @@ struct -- !query select ceiling(0) -- !query schema -struct +struct -- !query output 0 @@ -301,7 +301,7 @@ struct -- !query select ceiling(1) -- !query schema -struct +struct -- !query output 1 @@ -317,7 +317,7 @@ struct -- !query select ceiling(1234567890123456) -- !query schema -struct +struct -- !query output 1234567890123456 @@ -333,7 +333,7 @@ struct -- !query select ceiling(-0.10) -- !query schema -struct +struct -- !query output 0 @@ -389,7 +389,7 @@ true -- !query select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null) -- !query schema -struct<(7 % 2):int,(7 % 0):int,(0 % 2):int,(7 % CAST(NULL AS INT)):int,(CAST(NULL AS INT) % 2):int,(CAST(NULL AS DOUBLE) % CAST(NULL AS DOUBLE)):double> +struct -- !query output 1 NULL 0 NULL NULL NULL @@ -405,7 +405,7 @@ struct -- !query select CHAR_LENGTH('abc') -- !query schema -struct +struct -- !query output 3 @@ -413,7 +413,7 @@ struct -- !query select CHARACTER_LENGTH('abc') -- !query schema -struct +struct -- !query output 3 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out index 5efb58c7fc1b0..f7bba96738eab 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out @@ -381,8 +381,8 @@ struct<> org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. -Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(b.`four` AS BIGINT))] -Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))]; +Expression in where clause: [(sum(DISTINCT CAST((outer(a.`four`) + b.`four`) AS BIGINT)) = CAST(b.`four` AS BIGINT))] +Invalid expressions: [sum(DISTINCT CAST((outer(a.`four`) + b.`four`) AS BIGINT))]; -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index 1f2bd5795cf1e..ae1cb2f171704 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -42,7 +42,7 @@ struct<> -- !query -CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) +CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) USING PARQUET -- !query schema struct<> -- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index ed27317121623..e5a7c43c0a8ab 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -590,7 +590,7 @@ struct -- !query SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) -- !query schema -struct +struct -- !query output -2 @@ -598,7 +598,7 @@ struct -- !query SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) -- !query schema -struct +struct -- !query output -1 @@ -606,7 +606,7 @@ struct -- !query SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query schema -struct +struct -- !query output -1 @@ -670,7 +670,7 @@ true -- !query SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query schema -struct +struct -- !query output -1 @@ -750,7 +750,7 @@ struct -- !query SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) -- !query schema -struct +struct -- !query output 0 @@ -758,7 +758,7 @@ struct -- !query SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) -- !query schema -struct +struct -- !query output -1 @@ -766,7 +766,7 @@ struct -- !query SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) -- !query schema -struct +struct -- !query output -1 @@ -774,7 +774,7 @@ struct -- !query SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) -- !query schema -struct +struct -- !query output -2 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/insert.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/insert.sql.out index 1046d0ec86bbd..63ad74aac32ec 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/insert.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/insert.sql.out @@ -64,7 +64,7 @@ struct<> -- !query select col1, col2, char_length(col3) from inserttest -- !query schema -struct +struct -- !query output 30 50 10000 NULL 3 7 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index 65b6641983a1d..e59b9d5b63a40 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -4404,7 +4404,7 @@ struct<> -- !query SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round -- !query schema -struct +struct -- !query output -0.000001000000000000 0 0 -1 0 -5.499999000000000000 -5 -5 -6 -5 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out index 3b26d561d20c3..5f89c799498ac 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out @@ -627,7 +627,7 @@ struct<> -- !query SELECT substr(f1, 99995) from toasttest -- !query schema -struct +struct -- !query output 567890 567890 @@ -638,7 +638,7 @@ struct -- !query SELECT substr(f1, 99995, 10) from toasttest -- !query schema -struct +struct -- !query output 567890 567890 diff --git a/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out b/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out index f5ca1eff9f0c3..88fef8908638e 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out @@ -301,6 +301,17 @@ CREATE VIEW `default`.`view_SPARK_30302`( AS SELECT a, b FROM tbl +-- !query +SHOW CREATE TABLE view_SPARK_30302 +-- !query schema +struct +-- !query output +CREATE VIEW `default`.`view_SPARK_30302` ( + `aaa`, + `bbb`) +AS SELECT a, b FROM tbl + + -- !query DROP VIEW view_SPARK_30302 -- !query schema @@ -331,6 +342,18 @@ COMMENT 'This is a comment with \'quoted text\' for view' AS SELECT a, b FROM tbl +-- !query +SHOW CREATE TABLE view_SPARK_30302 +-- !query schema +struct +-- !query output +CREATE VIEW `default`.`view_SPARK_30302` ( + `aaa` COMMENT 'comment with \'quoted text\' for aaa', + `bbb`) +COMMENT 'This is a comment with \'quoted text\' for view' +AS SELECT a, b FROM tbl + + -- !query DROP VIEW view_SPARK_30302 -- !query schema @@ -363,30 +386,18 @@ TBLPROPERTIES ( AS SELECT a, b FROM tbl --- !query -DROP VIEW view_SPARK_30302 --- !query schema -struct<> --- !query output - - - --- !query -CREATE VIEW view_SPARK_30302 (aaa, bbb) -AS SELECT a, b FROM tbl --- !query schema -struct<> --- !query output - - - -- !query SHOW CREATE TABLE view_SPARK_30302 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -Hive view isn't supported by SHOW CREATE TABLE; +CREATE VIEW `default`.`view_SPARK_30302` ( + `aaa`, + `bbb`) +TBLPROPERTIES ( + 'a' = '1', + 'b' = '2') +AS SELECT a, b FROM tbl -- !query 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 501e185b07f7a..a95b02c7f7743 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 26 +-- Number of queries: 27 -- !query @@ -63,15 +63,9 @@ SHOW TABLES -- !query schema struct -- !query output -aggtest -arraydata -mapdata -onek show_t1 show_t2 show_t3 -tenk1 -testdata -- !query @@ -79,15 +73,9 @@ SHOW TABLES IN showdb -- !query schema struct -- !query output -aggtest -arraydata -mapdata -onek show_t1 show_t2 show_t3 -tenk1 -testdata -- !query @@ -119,6 +107,16 @@ show_t2 show_t3 +-- !query +SHOW TABLES IN showdb LIKE 'show_t*' +-- !query schema +struct +-- !query output +show_t1 +show_t2 +show_t3 + + -- !query SHOW TABLE EXTENDED LIKE 'show_t*' -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/show-views.sql.out b/sql/core/src/test/resources/sql-tests/results/show-views.sql.out new file mode 100644 index 0000000000000..d88790d8b5ec8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/show-views.sql.out @@ -0,0 +1,177 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 20 + + +-- !query +CREATE DATABASE showdb +-- !query schema +struct<> +-- !query output + + + +-- !query +USE showdb +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl(a STRING, b INT, c STRING, d STRING) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW view_1 AS SELECT * FROM tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW view_2 AS SELECT * FROM tbl WHERE c='a' +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE GLOBAL TEMP VIEW view_3 AS SELECT 1 as col1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW view_4(e INT) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW VIEWS +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS FROM showdb +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS IN showdb +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS IN global_temp +-- !query schema +struct +-- !query output +view_3 +view_4 + + +-- !query +SHOW VIEWS 'view_*' +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS LIKE 'view_1*|view_2*' +-- !query schema +struct +-- !query output +view_1 +view_2 + + +-- !query +SHOW VIEWS IN showdb 'view_*' +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS IN showdb LIKE 'view_*' +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS IN wrongdb LIKE 'view_*' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +Database 'wrongdb' not found; + + +-- !query +DROP VIEW global_temp.view_3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW view_4 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP DATABASE showdb CASCADE +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index 708dbb404c285..042d332bdb5c2 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -111,7 +111,7 @@ struct> -- !query SELECT substr('Spark SQL', 5) -- !query schema -struct +struct -- !query output k SQL @@ -119,7 +119,7 @@ k SQL -- !query SELECT substr('Spark SQL', -3) -- !query schema -struct +struct -- !query output SQL @@ -127,7 +127,7 @@ SQL -- !query SELECT substr('Spark SQL', 5, 1) -- !query schema -struct +struct -- !query output k diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/nested-not-in.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/nested-not-in.sql.out new file mode 100644 index 0000000000000..86f967eee1e0c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/nested-not-in.sql.out @@ -0,0 +1,332 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 21 + + +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", 10), + (200, "emp 2", NULL), + (300, "emp 3", 20), + (400, "emp 4", 30), + (500, "emp 5", NULL), + (600, "emp 6", 100), + (800, "emp 8", 70) +AS EMP(id, emp_name, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +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 schema +struct<> +-- !query output + + + +-- !query +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 schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW ADDRESS AS SELECT * FROM VALUES + (100, "emp 1", "addr1"), + (200, null, "addr2"), + (null, "emp 3", "addr3"), + (null, null, "addr4"), + (600, "emp 6", "addr6"), + (800, "emp 8", "addr8") +AS ADDRESS(id, emp_name, address) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW S1 AS SELECT * FROM VALUES + (null, null), (5, 5), (8, 8), (11, 11) AS s1(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW S2 AS SELECT * FROM VALUES + (7, 7), (8, 8), (11, 11), (null, null) AS s2(c, d) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 600 + OR id = 500 + OR dept_id NOT IN (SELECT dept_id + FROM emp) +-- !query schema +struct +-- !query output +500 NULL +600 100 + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 800 + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM emp)) +-- !query schema +struct +-- !query output +800 70 + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 100 + OR dept_id NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL) +-- !query schema +struct +-- !query output +100 10 + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 200 + OR (dept_id IS NOT NULL + AND dept_id + 100 NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL)) +-- !query schema +struct +-- !query output +100 10 +200 NULL +300 20 +400 30 +600 100 +800 70 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE emp_name IN (SELECT emp_name + FROM bonus) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)) +-- !query schema +struct +-- !query output +100 10 emp 1 +200 NULL emp 2 +300 20 emp 3 +400 30 emp 4 +500 NULL emp 5 +600 100 emp 6 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE EXISTS (SELECT emp_name + FROM bonus + WHERE emp.emp_name = bonus.emp_name) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)) +-- !query schema +struct +-- !query output +100 10 emp 1 +200 NULL emp 2 +300 20 emp 3 +400 30 emp 4 +500 NULL emp 5 +600 100 emp 6 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 +OR (id, emp_name) NOT IN (SELECT id, emp_name FROM address) +-- !query schema +struct +-- !query output +100 10 emp 1 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR (( id, emp_name ) NOT IN (SELECT id, + emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL) + AND id > 400 ) +-- !query schema +struct +-- !query output +100 10 emp 1 +500 NULL emp 5 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id) +-- !query schema +struct +-- !query output +100 10 emp 1 +200 NULL emp 2 +300 20 emp 3 +400 30 emp 4 +500 NULL emp 5 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE id NOT IN (SELECT id + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND id >= 400) + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id + AND id < 400) +-- !query schema +struct +-- !query output +100 10 emp 1 +200 NULL emp 2 +300 20 emp 3 +400 30 emp 4 +500 NULL emp 5 +600 100 emp 6 +800 70 emp 8 + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a NOT IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output +11 11 +8 8 + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a NOT IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output +5 5 + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a NOT IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output +11 11 +5 5 +8 8 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 index ec7ecf28754ef..d703d4e9112e9 100644 --- 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 @@ -109,7 +109,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses: -Aggregate [min(outer(t2a#x)) AS min(outer())#x] +Aggregate [min(outer(t2a#x)) AS min(outer(t2.`t2a`))#x] +- SubqueryAlias t3 +- Project [t3a#x, t3b#x, t3c#x] +- SubqueryAlias t3 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out index f841adf89612e..e47decbd33920 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out @@ -285,7 +285,7 @@ struct -- !query SELECT day( '1996-01-10') FROM t -- !query schema -struct +struct -- !query output 10 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out index 31353bdedc69f..b8c190beeae19 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out @@ -107,7 +107,7 @@ SELECT '1' + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST('1' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DOUBLE)' is of double type.; line 1 pos 7 -- !query @@ -698,7 +698,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + '1' FROM t struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST('1' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DOUBLE)' is of double type.; line 1 pos 7 -- !query @@ -790,7 +790,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t struct<> -- !query output org.apache.spark.sql.AnalysisException -cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST('1' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DOUBLE)' is of double type.; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index adf434b1bde95..76637bf578e6f 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -372,8 +372,8 @@ struct<> org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. -Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(CAST(udf(ansi_cast(four as string)) AS INT) AS BIGINT))] -Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))]; +Expression in where clause: [(sum(DISTINCT CAST((outer(a.`four`) + b.`four`) AS BIGINT)) = CAST(CAST(udf(ansi_cast(four as string)) AS INT) AS BIGINT))] +Invalid expressions: [sum(DISTINCT CAST((outer(a.`four`) + b.`four`) AS BIGINT))]; -- !query diff --git a/sql/core/src/test/resources/test-data/before_1582_date_v2_4.snappy.orc b/sql/core/src/test/resources/test-data/before_1582_date_v2_4.snappy.orc new file mode 100644 index 0000000000000..ebe01743b2e20 Binary files /dev/null and b/sql/core/src/test/resources/test-data/before_1582_date_v2_4.snappy.orc differ diff --git a/sql/core/src/test/resources/test-data/before_1582_date_v2_4.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_date_v2_4.snappy.parquet new file mode 100644 index 0000000000000..7d5cc12eefe04 Binary files /dev/null and b/sql/core/src/test/resources/test-data/before_1582_date_v2_4.snappy.parquet differ diff --git a/sql/core/src/test/resources/test-data/before_1582_timestamp_int96_v2_4.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_timestamp_int96_v2_4.snappy.parquet new file mode 100644 index 0000000000000..13254bd93a5e6 Binary files /dev/null and b/sql/core/src/test/resources/test-data/before_1582_timestamp_int96_v2_4.snappy.parquet differ diff --git a/sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4.snappy.parquet new file mode 100644 index 0000000000000..7d2b46e9bea41 Binary files /dev/null and b/sql/core/src/test/resources/test-data/before_1582_timestamp_micros_v2_4.snappy.parquet differ diff --git a/sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4.snappy.parquet b/sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4.snappy.parquet new file mode 100644 index 0000000000000..e9825455c2015 Binary files /dev/null and b/sql/core/src/test/resources/test-data/before_1582_timestamp_millis_v2_4.snappy.parquet differ diff --git a/sql/core/src/test/resources/test-data/before_1582_ts_v2_4.snappy.orc b/sql/core/src/test/resources/test-data/before_1582_ts_v2_4.snappy.orc new file mode 100644 index 0000000000000..af9ef040270ac Binary files /dev/null and b/sql/core/src/test/resources/test-data/before_1582_ts_v2_4.snappy.orc differ 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 81893534fc2a3..9baaaa9e07c0e 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 @@ -1122,4 +1122,46 @@ class CachedTableSuite extends QueryTest with SQLTestUtils assert(!spark.catalog.isCached("t1")) } } + + test("SPARK-30494 Fix the leak of cached data when replace an existing view") { + withTempView("tempView") { + spark.catalog.clearCache() + sql("create or replace temporary view tempView as select 1") + sql("cache table tempView") + assert(spark.sharedState.cacheManager.lookupCachedData(sql("select 1")).isDefined) + sql("create or replace temporary view tempView as select 1, 2") + assert(spark.sharedState.cacheManager.lookupCachedData(sql("select 1")).isEmpty) + sql("cache table tempView") + assert(spark.sharedState.cacheManager.lookupCachedData(sql("select 1, 2")).isDefined) + } + + withGlobalTempView("tempGlobalTempView") { + spark.catalog.clearCache() + sql("create or replace global temporary view tempGlobalTempView as select 1") + sql("cache table global_temp.tempGlobalTempView") + assert(spark.sharedState.cacheManager.lookupCachedData(sql("select 1")).isDefined) + sql("create or replace global temporary view tempGlobalTempView as select 1, 2") + assert(spark.sharedState.cacheManager.lookupCachedData(sql("select 1")).isEmpty) + sql("cache table global_temp.tempGlobalTempView") + assert(spark.sharedState.cacheManager.lookupCachedData(sql("select 1, 2")).isDefined) + } + + withView("view1") { + spark.catalog.clearCache() + sql("create or replace view view1 as select 1") + sql("cache table view1") + sql("create or replace view view1 as select 1, 2") + sql("cache table view1") + // the cached plan of persisted view likes below, + // we cannot use the same assertion of temp view. + // SubqueryAlias + // | + // + View + // | + // + Project[1 AS 1] + spark.sharedState.cacheManager.uncacheQuery(spark.table("view1"), cascade = false) + // make sure there is no cached data leak + assert(spark.sharedState.cacheManager.isEmpty) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala index c3dbbb325d842..36989efbe870d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.commons.math3.stat.inference.ChiSquareTest +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -27,7 +28,8 @@ class ConfigBehaviorSuite extends QueryTest with SharedSparkSession { import testImplicits._ - test("SPARK-22160 spark.sql.execution.rangeExchange.sampleSizePerPartition") { + test("SPARK-22160 spark.sql.execution.rangeExchange.sampleSizePerPartition", + DisableAdaptiveExecution("Post shuffle partition number can be different")) { // In this test, we run a sort and compute the histogram for partition size post shuffle. // With a high sample count, the partition size should be more evenly distributed, and has a // low chi-sq test value. @@ -53,11 +55,8 @@ class ConfigBehaviorSuite extends QueryTest with SharedSparkSession { dist) } - // When enable AQE, the post partition number is changed. // And the ChiSquareTest result is also need updated. So disable AQE. - withSQLConf( - SQLConf.SHUFFLE_PARTITIONS.key -> numPartitions.toString, - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> numPartitions.toString) { // The default chi-sq value should be low assert(computeChiSquareTest() < 100) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index a186775528b3e..800e294cca8c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -238,4 +238,16 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { spark.range(1).select(schema_of_csv(input)), Seq(Row("struct<_c0:double,_c1:int>"))) } + + test("optional datetime parser does not affect csv time formatting") { + val s = "2015-08-26 12:34:46" + def toDF(p: String): DataFrame = sql( + s""" + |SELECT + | to_csv( + | named_struct('time', timestamp'$s'), map('timestampFormat', "$p") + | ) + | """.stripMargin) + checkAnswer(toDF("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), toDF("yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index a613c33b6c876..ac98d3f0c7095 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -490,6 +491,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { withSQLConf(SQLConf.LEGACY_SIZE_OF_NULL.key -> "false") { testSizeOfArray(sizeOfNull = null) } + // size(null) should return null under ansi mode. + withSQLConf( + SQLConf.LEGACY_SIZE_OF_NULL.key -> "true", + SQLConf.ANSI_ENABLED.key -> "true") { + testSizeOfArray(sizeOfNull = null) + } } test("dataframe arrays_zip function") { @@ -569,6 +576,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { withSQLConf(SQLConf.LEGACY_SIZE_OF_NULL.key -> "false") { testSizeOfMap(sizeOfNull = null) } + // size(null) should return null under ansi mode. + withSQLConf( + SQLConf.LEGACY_SIZE_OF_NULL.key -> "true", + SQLConf.ANSI_ENABLED.key -> "true") { + testSizeOfMap(sizeOfNull = null) + } } test("map_keys/map_values function") { @@ -1016,7 +1029,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Timestamp.valueOf("2018-01-01 12:00:00"), Timestamp.valueOf("2018-01-02 00:00:00"))))) - DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone("UTC")) { + DateTimeTestUtils.withDefaultTimeZone(UTC) { checkAnswer( spark.sql("select sequence(" + " cast('2018-01-01' as date)" + @@ -1520,6 +1533,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { assert(e.getMessage.contains("string, binary or array")) } + test("SPARK-31227: Non-nullable null type should not coerce to nullable type in concat") { + val actual = spark.range(1).selectExpr("concat(array(), array(1)) as arr") + val expected = spark.range(1).selectExpr("array(1) as arr") + checkAnswer(actual, expected) + assert(actual.schema === expected.schema) + } + test("flatten function") { // Test cases with a primitive type val intDF = Seq( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index c7545bcad8962..6b772e53ac184 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -17,10 +17,14 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike, LeftOuter, RightOuter} -import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, Filter, HintInfo, Join, JoinHint, LogicalPlan, Project} +import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -322,4 +326,96 @@ class DataFrameJoinSuite extends QueryTest } } } + + test("Supports multi-part names for broadcast hint resolution") { + val (table1Name, table2Name) = ("t1", "t2") + + withTempDatabase { dbName => + withTable(table1Name, table2Name) { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + spark.range(50).write.saveAsTable(s"$dbName.$table1Name") + spark.range(100).write.saveAsTable(s"$dbName.$table2Name") + + def checkIfHintApplied(df: DataFrame): Unit = { + val sparkPlan = df.queryExecution.executedPlan + val broadcastHashJoins = sparkPlan.collect { case p: BroadcastHashJoinExec => p } + assert(broadcastHashJoins.size == 1) + val broadcastExchanges = broadcastHashJoins.head.collect { + case p: BroadcastExchangeExec => p + } + assert(broadcastExchanges.size == 1) + val tables = broadcastExchanges.head.collect { + case FileSourceScanExec(_, _, _, _, _, _, Some(tableIdent)) => tableIdent + } + assert(tables.size == 1) + assert(tables.head === TableIdentifier(table1Name, Some(dbName))) + } + + def checkIfHintNotApplied(df: DataFrame): Unit = { + val sparkPlan = df.queryExecution.executedPlan + val broadcastHashJoins = sparkPlan.collect { case p: BroadcastHashJoinExec => p } + assert(broadcastHashJoins.isEmpty) + } + + def sqlTemplate(tableName: String, hintTableName: String): DataFrame = { + sql(s"SELECT /*+ BROADCASTJOIN($hintTableName) */ * " + + s"FROM $tableName, $dbName.$table2Name " + + s"WHERE $tableName.id = $table2Name.id") + } + + def dfTemplate(tableName: String, hintTableName: String): DataFrame = { + spark.table(tableName).join(spark.table(s"$dbName.$table2Name"), "id") + .hint("broadcast", hintTableName) + } + + sql(s"USE $dbName") + + checkIfHintApplied(sqlTemplate(table1Name, table1Name)) + checkIfHintApplied(sqlTemplate(s"$dbName.$table1Name", s"$dbName.$table1Name")) + checkIfHintApplied(sqlTemplate(s"$dbName.$table1Name", table1Name)) + checkIfHintNotApplied(sqlTemplate(table1Name, s"$dbName.$table1Name")) + + checkIfHintApplied(dfTemplate(table1Name, table1Name)) + checkIfHintApplied(dfTemplate(s"$dbName.$table1Name", s"$dbName.$table1Name")) + checkIfHintApplied(dfTemplate(s"$dbName.$table1Name", table1Name)) + checkIfHintApplied(dfTemplate(table1Name, s"$dbName.$table1Name")) + checkIfHintApplied(dfTemplate(table1Name, + s"${CatalogManager.SESSION_CATALOG_NAME}.$dbName.$table1Name")) + + withView("tv") { + sql(s"CREATE VIEW tv AS SELECT * FROM $dbName.$table1Name") + checkIfHintApplied(sqlTemplate("tv", "tv")) + checkIfHintNotApplied(sqlTemplate("tv", s"$dbName.tv")) + + checkIfHintApplied(dfTemplate("tv", "tv")) + checkIfHintApplied(dfTemplate("tv", s"$dbName.tv")) + } + } + } + } + } + + test("The same table name exists in two databases for broadcast hint resolution") { + val (db1Name, db2Name) = ("db1", "db2") + + withDatabase(db1Name, db2Name) { + withTable("t") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + sql(s"CREATE DATABASE $db1Name") + sql(s"CREATE DATABASE $db2Name") + spark.range(1).write.saveAsTable(s"$db1Name.t") + spark.range(1).write.saveAsTable(s"$db2Name.t") + + // Checks if a broadcast hint applied in both sides + val statement = s"SELECT /*+ BROADCASTJOIN(t) */ * FROM $db1Name.t, $db2Name.t " + + s"WHERE $db1Name.t.id = $db2Name.t.id" + sql(statement).queryExecution.optimizedPlan match { + case Join(_, _, _, _, JoinHint(Some(HintInfo(Some(BROADCAST))), + Some(HintInfo(Some(BROADCAST))))) => + case _ => fail("broadcast hint not found in both tables") + } + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index 51c6a835d58d6..51d861ec69b23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -258,7 +258,7 @@ class DataFramePivotSuite extends QueryTest with SharedSparkSession { val ts = "2012-12-31 16:00:10.011" val tsWithZone = "2013-01-01 00:00:10.011" - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { val df = Seq(java.sql.Timestamp.valueOf(ts)).toDF("a").groupBy("a").pivot("a").count() val expected = StructType( StructField("a", TimestampType) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index 394bad751b5ce..19601726fa393 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -126,6 +126,32 @@ class DataFrameStatSuite extends QueryTest with SharedSparkSession { assert(math.abs(corr3 - 0.95723391394758572) < 1e-12) } + test("SPARK-30532 stat functions to understand fully-qualified column name") { + val df1 = spark.sparkContext.parallelize(0 to 10).toDF("num").as("table1") + val df2 = spark.sparkContext.parallelize(0 to 10).toDF("num").as("table2") + val dfx = df2.crossJoin(df1) + + assert(dfx.stat.corr("table1.num", "table2.num") != 0.0) + assert(dfx.stat.cov("table1.num", "table2.num") != 0.0) + assert(dfx.stat.approxQuantile("table1.num", Array(0.1), 0.0).length == 1) + assert(dfx.stat.approxQuantile("table2.num", Array(0.1), 0.0).length == 1) + assert(dfx.stat.freqItems(Array("table1.num", "table2.num")).collect()(0).length == 2) + + // this should throw "Reference 'num' is ambiguous" + intercept[AnalysisException] { + dfx.stat.freqItems(Array("num")) + } + intercept[AnalysisException] { + dfx.stat.approxQuantile("num", Array(0.1), 0.0) + } + intercept[AnalysisException] { + dfx.stat.cov("num", "num") + } + intercept[AnalysisException] { + dfx.stat.corr("num", "num") + } + } + test("covariance") { val df = Seq.tabulate(10)(i => (i, 2.0 * i, toLetter(i))).toDF("singles", "doubles", "letters") 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 e74d5532849dd..f797290028e6f 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 @@ -43,6 +43,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} import org.apache.spark.sql.test.SQLTestData.{DecimalData, NullStrings, TestData2} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom @@ -109,6 +110,31 @@ class DataFrameSuite extends QueryTest dfAlias.col("t2.c") } + test("simple explode") { + val df = Seq(Tuple1("a b c"), Tuple1("d e")).toDF("words") + + checkAnswer( + df.explode("words", "word") { word: String => word.split(" ").toSeq }.select('word), + Row("a") :: Row("b") :: Row("c") :: Row("d") ::Row("e") :: Nil + ) + } + + test("explode") { + val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDF("number", "letters") + val df2 = + df.explode('letters) { + case Row(letters: String) => letters.split(" ").map(Tuple1(_)).toSeq + } + + checkAnswer( + df2 + .select('_1 as 'letter, 'number) + .groupBy('letter) + .agg(countDistinct('number)), + Row("a", 3) :: Row("b", 2) :: Row("c", 1) :: Nil + ) + } + test("Star Expansion - CreateStruct and CreateArray") { val structDf = testData2.select("a", "b").as("record") // CreateStruct and CreateArray in aggregateExpressions @@ -185,6 +211,27 @@ class DataFrameSuite extends QueryTest } } + test("Star Expansion - ds.explode should fail with a meaningful message if it takes a star") { + val df = Seq(("1", "1,2"), ("2", "4"), ("3", "7,8,9")).toDF("prefix", "csv") + val e = intercept[AnalysisException] { + df.explode($"*") { case Row(prefix: String, csv: String) => + csv.split(",").map(v => Tuple1(prefix + ":" + v)).toSeq + }.queryExecution.assertAnalyzed() + } + assert(e.getMessage.contains("Invalid usage of '*' in explode/json_tuple/UDTF")) + + checkAnswer( + df.explode('prefix, 'csv) { case Row(prefix: String, csv: String) => + csv.split(",").map(v => Tuple1(prefix + ":" + v)).toSeq + }, + Row("1", "1,2", "1:1") :: + Row("1", "1,2", "1:2") :: + Row("2", "4", "2:4") :: + Row("3", "7,8,9", "3:7") :: + Row("3", "7,8,9", "3:8") :: + Row("3", "7,8,9", "3:9") :: Nil) + } + test("Star Expansion - explode should fail with a meaningful message if it takes a star") { val df = Seq(("1,2"), ("4"), ("7,8,9")).toDF("csv") val e = intercept[AnalysisException] { @@ -1167,7 +1214,7 @@ class DataFrameSuite extends QueryTest |""".stripMargin assert(df.showString(1, truncate = 0) === expectedAnswer) - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { val expectedAnswer = """+----------+-------------------+ ||d |ts | @@ -1188,7 +1235,7 @@ class DataFrameSuite extends QueryTest " ts | 2016-12-01 00:00:00 \n" assert(df.showString(1, truncate = 0, vertical = true) === expectedAnswer) - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { val expectedAnswer = "-RECORD 0------------------\n" + " d | 2016-12-01 \n" + @@ -2306,6 +2353,11 @@ class DataFrameSuite extends QueryTest } assert(e.getMessage.contains("Table or view not found:")) } + + test("CalendarInterval reflection support") { + val df = Seq((1, new CalendarInterval(1, 2, 3))).toDF("a", "b") + checkAnswer(df.selectExpr("b"), Row(new CalendarInterval(1, 2, 3))) + } } case class GroupByKey(a: Int, b: Int) 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 b4ed4ec28ceb0..af65957691b37 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 @@ -1222,14 +1222,6 @@ class DatasetSuite extends QueryTest assert(result == Set(ClassData("a", 1) -> null, ClassData("b", 2) -> ClassData("x", 2))) } - test("better error message when use java reserved keyword as field name") { - val e = intercept[UnsupportedOperationException] { - Seq(InvalidInJava(1)).toDS() - } - assert(e.getMessage.contains( - "`abstract` is a reserved keyword and cannot be used as field name")) - } - test("Dataset should support flat input object to be null") { checkDataset(Seq("a", null).toDS(), "a", null) } @@ -1964,8 +1956,6 @@ case class ClassNullableData(a: String, b: Integer) case class NestedStruct(f: ClassData) case class DeepNestedStruct(f: NestedStruct) -case class InvalidInJava(`abstract`: Int) - /** * A class used to test serialization using encoders. This class throws exceptions when using * Java serialization -- so the only way it can be "serialized" is through our encoders. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 3865012c97ae7..1d5140b355999 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -24,6 +24,7 @@ import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{CEST, LA} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -761,12 +762,12 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") ).toDF("a", "b") checkAnswer( - df.select(from_utc_timestamp(col("a"), "PST")), + df.select(from_utc_timestamp(col("a"), LA.getId)), Seq( Row(Timestamp.valueOf("2015-07-23 17:00:00")), Row(Timestamp.valueOf("2015-07-24 17:00:00")))) checkAnswer( - df.select(from_utc_timestamp(col("b"), "PST")), + df.select(from_utc_timestamp(col("b"), LA.getId)), Seq( Row(Timestamp.valueOf("2015-07-23 17:00:00")), Row(Timestamp.valueOf("2015-07-24 17:00:00")))) @@ -774,8 +775,8 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { test("from_utc_timestamp with column zone") { val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "CET"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "PST") + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", CEST.getId), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", LA.getId) ).toDF("a", "b", "c") checkAnswer( df.select(from_utc_timestamp(col("a"), col("c"))), @@ -804,12 +805,12 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") ).toDF("a", "b") checkAnswer( - df.select(to_utc_timestamp(col("a"), "PST")), + df.select(to_utc_timestamp(col("a"), LA.getId)), Seq( Row(Timestamp.valueOf("2015-07-24 07:00:00")), Row(Timestamp.valueOf("2015-07-25 07:00:00")))) checkAnswer( - df.select(to_utc_timestamp(col("b"), "PST")), + df.select(to_utc_timestamp(col("b"), LA.getId)), Seq( Row(Timestamp.valueOf("2015-07-24 07:00:00")), Row(Timestamp.valueOf("2015-07-25 07:00:00")))) @@ -817,8 +818,8 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { test("to_utc_timestamp with column zone") { val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "PST"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "CET") + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", LA.getId), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", CEST.getId) ).toDF("a", "b", "c") checkAnswer( df.select(to_utc_timestamp(col("a"), col("c"))), @@ -849,10 +850,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { } test("SPARK-30752: convert time zones on a daylight saving day") { - val systemTz = "PST" + val systemTz = LA.getId val sessionTz = "UTC" val fromTz = "Asia/Hong_Kong" - val fromTs = "2019-11-03T12:00:00" // daylight saving date in PST + val fromTs = "2019-11-03T12:00:00" // daylight saving date in America/Los_Angeles val utsTs = "2019-11-03T04:00:00" val defaultTz = TimeZone.getDefault try { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedAPISuite.scala new file mode 100644 index 0000000000000..25b8849d61248 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedAPISuite.scala @@ -0,0 +1,220 @@ +/* + * 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.sql.functions._ +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} + +class DeprecatedAPISuite extends QueryTest with SharedSparkSession { + import MathFunctionsTestData.DoubleData + import testImplicits._ + + private lazy val doubleData = (1 to 10).map(i => DoubleData(i * 0.2 - 1, i * -0.2 + 1)).toDF() + + private def testOneToOneMathFunction[ + @specialized(Int, Long, Float, Double) T, + @specialized(Int, Long, Float, Double) U]( + c: Column => Column, + f: T => U): Unit = { + checkAnswer( + doubleData.select(c('a)), + (1 to 10).map(n => Row(f((n * 0.2 - 1).asInstanceOf[T]))) + ) + + checkAnswer( + doubleData.select(c('b)), + (1 to 10).map(n => Row(f((-n * 0.2 + 1).asInstanceOf[T]))) + ) + + checkAnswer( + doubleData.select(c(lit(null))), + (1 to 10).map(_ => Row(null)) + ) + } + + test("functions.toDegrees") { + testOneToOneMathFunction(toDegrees, math.toDegrees) + withView("t") { + val df = Seq(0, 1, 1.5).toDF("a") + df.createOrReplaceTempView("t") + + checkAnswer( + sql("SELECT degrees(0), degrees(1), degrees(1.5)"), + Seq(0).toDF().select(toDegrees(lit(0)), toDegrees(lit(1)), toDegrees(lit(1.5))) + ) + checkAnswer( + sql("SELECT degrees(a) FROM t"), + df.select(toDegrees("a")) + ) + } + } + + test("functions.toRadians") { + testOneToOneMathFunction(toRadians, math.toRadians) + withView("t") { + val df = Seq(0, 1, 1.5).toDF("a") + df.createOrReplaceTempView("t") + + checkAnswer( + sql("SELECT radians(0), radians(1), radians(1.5)"), + Seq(0).toDF().select(toRadians(lit(0)), toRadians(lit(1)), toRadians(lit(1.5))) + ) + checkAnswer( + sql("SELECT radians(a) FROM t"), + df.select(toRadians("a")) + ) + } + } + + test("functions.approxCountDistinct") { + withView("t") { + val df = Seq(0, 1, 2).toDF("a") + df.createOrReplaceTempView("t") + checkAnswer( + sql("SELECT approx_count_distinct(a) FROM t"), + df.select(approxCountDistinct("a"))) + } + } + + test("functions.monotonicallyIncreasingId") { + // Make sure we have 2 partitions, each with 2 records. + val df = sparkContext.parallelize(Seq[Int](), 2).mapPartitions { _ => + Iterator(Tuple1(1), Tuple1(2)) + }.toDF("a") + checkAnswer( + df.select(monotonicallyIncreasingId(), expr("monotonically_increasing_id()")), + Row(0L, 0L) :: + Row(1L, 1L) :: + Row((1L << 33) + 0L, (1L << 33) + 0L) :: + Row((1L << 33) + 1L, (1L << 33) + 1L) :: Nil + ) + } + + test("Column.!==") { + val nullData = Seq( + (Some(1), Some(1)), (Some(1), Some(2)), (Some(1), None), (None, None)).toDF("a", "b") + checkAnswer( + nullData.filter($"b" !== 1), + Row(1, 2) :: Nil) + + checkAnswer(nullData.filter($"b" !== null), Nil) + + checkAnswer( + nullData.filter($"a" !== $"b"), + Row(1, 2) :: Nil) + } + + test("Dataset.registerTempTable") { + withTempView("t") { + Seq(1).toDF().registerTempTable("t") + assert(spark.catalog.tableExists("t")) + } + } + + test("SQLContext.setActive/clearActive") { + val sc = spark.sparkContext + val sqlContext = new SQLContext(sc) + SQLContext.setActive(sqlContext) + assert(SparkSession.getActiveSession === Some(spark)) + SQLContext.clearActive() + assert(SparkSession.getActiveSession === None) + } + + test("SQLContext.applySchema") { + val rowRdd = sparkContext.parallelize(Seq(Row("Jack", 20), Row("Marry", 18))) + val schema = StructType(StructField("name", StringType, false) :: + StructField("age", IntegerType, true) :: Nil) + val sqlContext = spark.sqlContext + checkAnswer(sqlContext.applySchema(rowRdd, schema), Row("Jack", 20) :: Row("Marry", 18) :: Nil) + checkAnswer(sqlContext.applySchema(rowRdd.toJavaRDD(), schema), + Row("Jack", 20) :: Row("Marry", 18) :: Nil) + } + + test("SQLContext.parquetFile") { + val sqlContext = spark.sqlContext + withTempDir { dir => + val parquetFile = s"${dir.toString}/${System.currentTimeMillis()}" + val expectDF = spark.range(10).toDF() + expectDF.write.parquet(parquetFile) + val parquetDF = sqlContext.parquetFile(parquetFile) + checkAnswer(parquetDF, expectDF) + } + } + + test("SQLContext.jsonFile") { + val sqlContext = spark.sqlContext + withTempDir { dir => + val jsonFile = s"${dir.toString}/${System.currentTimeMillis()}" + val expectDF = spark.range(10).toDF() + expectDF.write.json(jsonFile) + var jsonDF = sqlContext.jsonFile(jsonFile) + checkAnswer(jsonDF, expectDF) + assert(jsonDF.schema === expectDF.schema.asNullable) + + var schema = expectDF.schema + jsonDF = sqlContext.jsonFile(jsonFile, schema) + checkAnswer(jsonDF, expectDF) + assert(jsonDF.schema === schema.asNullable) + + jsonDF = sqlContext.jsonFile(jsonFile, 0.9) + checkAnswer(jsonDF, expectDF) + + val jsonRDD = sparkContext.parallelize(Seq("{\"name\":\"Jack\",\"age\":20}", + "{\"name\":\"Marry\",\"age\":18}")) + jsonDF = sqlContext.jsonRDD(jsonRDD) + checkAnswer(jsonDF, Row(18, "Marry") :: Row(20, "Jack") :: Nil) + jsonDF = sqlContext.jsonRDD(jsonRDD.toJavaRDD()) + checkAnswer(jsonDF, Row(18, "Marry") :: Row(20, "Jack") :: Nil) + + schema = StructType(StructField("name", StringType, false) :: + StructField("age", IntegerType, false) :: Nil) + jsonDF = sqlContext.jsonRDD(jsonRDD, schema) + checkAnswer(jsonDF, Row("Jack", 20) :: Row("Marry", 18) :: Nil) + jsonDF = sqlContext.jsonRDD(jsonRDD.toJavaRDD(), schema) + checkAnswer(jsonDF, Row("Jack", 20) :: Row("Marry", 18) :: Nil) + + + jsonDF = sqlContext.jsonRDD(jsonRDD, 0.9) + checkAnswer(jsonDF, Row(18, "Marry") :: Row(20, "Jack") :: Nil) + jsonDF = sqlContext.jsonRDD(jsonRDD.toJavaRDD(), 0.9) + checkAnswer(jsonDF, Row(18, "Marry") :: Row(20, "Jack") :: Nil) + } + } + + test("SQLContext.load") { + withTempDir { dir => + val path = s"${dir.toString}/${System.currentTimeMillis()}" + val expectDF = spark.range(10).toDF() + expectDF.write.parquet(path) + val sqlContext = spark.sqlContext + + var loadDF = sqlContext.load(path) + checkAnswer(loadDF, expectDF) + + loadDF = sqlContext.load(path, "parquet") + checkAnswer(loadDF, expectDF) + + loadDF = sqlContext.load("parquet", Map("path" -> path)) + checkAnswer(loadDF, expectDF) + + loadDF = sqlContext.load("parquet", expectDF.schema, Map("path" -> path)) + checkAnswer(loadDF, expectDF) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index baa9f5ecafc68..cdf9ea4b31ee7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1234,7 +1234,7 @@ abstract class DynamicPartitionPruningSuiteBase val plan = df.queryExecution.executedPlan val countSubqueryBroadcasts = - plan.collectInPlanAndSubqueries({ case _: SubqueryBroadcastExec => 1 }).sum + plan.collectWithSubqueries({ case _: SubqueryBroadcastExec => 1 }).sum assert(countSubqueryBroadcasts == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index b591705274110..1a35e5b729c00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -18,26 +18,15 @@ package org.apache.spark.sql import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType -class ExplainSuite extends QueryTest with SharedSparkSession { +class ExplainSuite extends QueryTest with SharedSparkSession with DisableAdaptiveExecutionSuite { import testImplicits._ - var originalValue: String = _ - protected override def beforeAll(): Unit = { - super.beforeAll() - originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") - } - - protected override def afterAll(): Unit = { - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) - super.afterAll() - } - private def getNormalizedExplain(df: DataFrame, mode: ExplainMode): String = { val output = new java.io.ByteArrayOutputStream() Console.withOut(output) { @@ -116,8 +105,8 @@ class ExplainSuite extends QueryTest with SharedSparkSession { // plan should show the rewritten aggregate expression. val df = sql("SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k") checkKeywordsExistsInExplain(df, - "Aggregate [k#x], [k#x, min(v#x) AS every(v)#x, max(v#x) AS some(v)#x, " + - "max(v#x) AS any(v)#x]") + "Aggregate [k#x], [k#x, every(v#x) AS every(v)#x, some(v#x) AS some(v)#x, " + + "any(v#x) AS any(v)#x]") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index c870958128483..cb410b4f0d7dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -842,6 +842,46 @@ class FileBasedDataSourceSuite extends QueryTest } } } + + test("SPARK-31116: Select nested schema with case insensitive mode") { + // This test case failed at only Parquet. ORC is added for test coverage parity. + Seq("orc", "parquet").foreach { format => + Seq("true", "false").foreach { nestedSchemaPruningEnabled => + withSQLConf( + SQLConf.CASE_SENSITIVE.key -> "false", + SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> nestedSchemaPruningEnabled) { + withTempPath { dir => + val path = dir.getCanonicalPath + + // Prepare values for testing nested parquet data + spark + .range(1L) + .selectExpr("NAMED_STRUCT('lowercase', id, 'camelCase', id + 1) AS StructColumn") + .write + .format(format) + .save(path) + + val exactSchema = "StructColumn struct" + + checkAnswer(spark.read.schema(exactSchema).format(format).load(path), Row(Row(0, 1))) + + // In case insensitive manner, parquet's column cases are ignored + val innerColumnCaseInsensitiveSchema = + "StructColumn struct" + checkAnswer( + spark.read.schema(innerColumnCaseInsensitiveSchema).format(format).load(path), + Row(Row(0, 1))) + + val rootColumnCaseInsensitiveSchema = + "structColumn struct" + checkAnswer( + spark.read.schema(rootColumnCaseInsensitiveSchema).format(format).load(path), + Row(Row(0, 1))) + } + } + } + } + } } object TestingUDT { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index 51150a1b38b49..4a4504a075060 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -337,7 +337,7 @@ object IntegratedUDFTestUtils extends SQLHelper { input.toString }, StringType, - inputSchemas = Seq.fill(1)(None), + inputEncoders = Seq.fill(1)(None), name = Some(name)) { override def apply(exprs: Column*): Column = { 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 65e1dde8ae141..5c2e3edda5510 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 @@ -710,4 +710,15 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { Seq(Row("string"))) } + test("optional datetime parser does not affect json time formatting") { + val s = "2015-08-26 12:34:46" + def toDF(p: String): DataFrame = sql( + s""" + |SELECT + | to_json( + | named_struct('time', timestamp'$s'), map('timestampFormat', "$p") + | ) + | """.stripMargin) + checkAnswer(toDF("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), toDF("yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala index 956bd7861d99d..a9f443be69cb2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.io.File import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.sql.execution.adaptive.AdaptiveTestUtils.assertExceptionMessage import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -55,8 +56,8 @@ abstract class MetadataCacheSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkException] { df.count() } - assert(e.getMessage.contains("FileNotFoundException")) - assert(e.getMessage.contains("recreating the Dataset/DataFrame involved")) + assertExceptionMessage(e, "FileNotFoundException") + assertExceptionMessage(e, "recreating the Dataset/DataFrame involved") } } } @@ -84,8 +85,8 @@ class MetadataCacheV1Suite extends MetadataCacheSuite { val e = intercept[SparkException] { sql("select count(*) from view_refresh").first() } - assert(e.getMessage.contains("FileNotFoundException")) - assert(e.getMessage.contains("REFRESH")) + assertExceptionMessage(e, "FileNotFoundException") + assertExceptionMessage(e, "REFRESH") // Refresh and we should be able to read it again. spark.catalog.refreshTable("view_refresh") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala index 5ab06b1ebebf6..16edf35bb99f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala @@ -34,9 +34,11 @@ class MiscFunctionsSuite extends QueryTest with SharedSparkSession { } test("version") { + val df = sql("SELECT version()") checkAnswer( - Seq("").toDF("a").selectExpr("version()"), + df, Row(SPARK_VERSION_SHORT + " " + SPARK_REVISION)) + assert(df.schema.fieldNames === Seq("version()")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index aab2ae4afc7f5..a1799829932b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -24,14 +24,32 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} +@deprecated("This suite is deprecated to silent compiler deprecation warnings", "2.0.0") class SQLContextSuite extends SparkFunSuite with SharedSparkContext { object DummyRule extends Rule[LogicalPlan] { def apply(p: LogicalPlan): LogicalPlan = p } + test("getOrCreate instantiates SQLContext") { + val sqlContext = SQLContext.getOrCreate(sc) + assert(sqlContext != null, "SQLContext.getOrCreate returned null") + assert(SQLContext.getOrCreate(sc).eq(sqlContext), + "SQLContext created by SQLContext.getOrCreate not returned by SQLContext.getOrCreate") + } + + test("getOrCreate return the original SQLContext") { + val sqlContext = SQLContext.getOrCreate(sc) + val newSession = sqlContext.newSession() + assert(SQLContext.getOrCreate(sc).eq(sqlContext), + "SQLContext.getOrCreate after explicitly created SQLContext did not return the context") + SparkSession.setActiveSession(newSession.sparkSession) + assert(SQLContext.getOrCreate(sc).eq(newSession), + "SQLContext.getOrCreate after explicitly setActive() did not return the active context") + } + test("Sessions of SQLContext") { - val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext + val sqlContext = SQLContext.getOrCreate(sc) val session1 = sqlContext.newSession() val session2 = sqlContext.newSession() @@ -59,13 +77,13 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { } test("Catalyst optimization passes are modifiable at runtime") { - val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext + val sqlContext = SQLContext.getOrCreate(sc) sqlContext.experimental.extraOptimizations = Seq(DummyRule) assert(sqlContext.sessionState.optimizer.batches.flatMap(_.rules).contains(DummyRule)) } test("get all tables") { - val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext + val sqlContext = SQLContext.getOrCreate(sc) val df = sqlContext.range(10) df.createOrReplaceTempView("listtablessuitetable") assert( @@ -82,7 +100,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { } test("getting all tables with a database name has no impact on returned table names") { - val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext + val sqlContext = SQLContext.getOrCreate(sc) val df = sqlContext.range(10) df.createOrReplaceTempView("listtablessuitetable") assert( @@ -99,7 +117,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { } test("query the returned DataFrame of tables") { - val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext + val sqlContext = SQLContext.getOrCreate(sc) val df = sqlContext.range(10) df.createOrReplaceTempView("listtablessuitetable") @@ -109,7 +127,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { StructField("isTemporary", BooleanType, false) :: Nil) Seq(sqlContext.tables(), sqlContext.sql("SHOW TABLes")).foreach { - tableDF => + case tableDF => assert(expectedSchema === tableDF.schema) tableDF.createOrReplaceTempView("tables") 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 823e432ba4b97..e199dcc33af64 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 @@ -181,6 +181,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark parFuncs.foreach { funcId => // Examples can change settings. We clone the session to prevent tests clashing. val clonedSpark = spark.cloneSession() + // Coalescing partitions can change result order, so disable it. + clonedSpark.sessionState.conf.setConf(SQLConf.COALESCE_PARTITIONS_ENABLED, false) val info = clonedSpark.sessionState.catalog.lookupFunctionInfo(funcId) val className = info.getClassName if (!ignoreSet.contains(className)) { @@ -3487,6 +3489,22 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } } + + test("SPARK-31166: UNION map and other maps should not fail") { + checkAnswer( + sql("(SELECT map()) UNION ALL (SELECT map(1, 2))"), + Seq(Row(Map[Int, Int]()), Row(Map(1 -> 2)))) + } + + test("SPARK-31242: clone SparkSession should respect sessionInitWithConfigDefaults") { + // Note, only the conf explicitly set in SparkConf(e.g. in SharedSparkSessionBase) would cause + // problem before the fix. + withSQLConf(SQLConf.CODEGEN_FALLBACK.key -> "true") { + val cloned = spark.cloneSession() + SparkSession.setActiveSession(cloned) + assert(SQLConf.get.getConf(SQLConf.CODEGEN_FALLBACK) === true) + } + } } case class Foo(bar: Option[String]) 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 6c661667825b2..bc9b4d85aacdb 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 @@ -25,12 +25,14 @@ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} +import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND +import org.apache.spark.sql.execution.{SQLExecution, WholeStageCodegenExec} import org.apache.spark.sql.execution.HiveResult.hiveResultString -import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -256,20 +258,23 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { def splitWithSemicolon(seq: Seq[String]) = { seq.mkString("\n").split("(?<=[^\\\\]);") } - val input = fileToString(new File(testCase.inputFile)) - val (comments, code) = input.split("\n").partition { line => + def splitCommentsAndCodes(input: String) = input.split("\n").partition { line => val newLine = line.trim newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER") } + val input = fileToString(new File(testCase.inputFile)) + + val (comments, code) = splitCommentsAndCodes(input) + // If `--IMPORT` found, load code from another test case file, then insert them // into the head in this test. val importedTestCaseName = comments.filter(_.startsWith("--IMPORT ")).map(_.substring(9)) val importedCode = importedTestCaseName.flatMap { testCaseName => listTestCases.find(_.name == testCaseName).map { testCase => val input = fileToString(new File(testCase.inputFile)) - val (_, code) = input.split("\n").partition(_.trim.startsWith("--")) + val (_, code) = splitCommentsAndCodes(input) code } }.flatten @@ -358,7 +363,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // Create a local SparkSession to have stronger isolation between different test cases. // This does not isolate catalog changes. val localSparkSession = spark.newSession() - loadTestData(localSparkSession) testCase match { case udfTestCase: UDFTest => @@ -567,14 +571,20 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { } /** Load built-in test tables into the SparkSession. */ - private def loadTestData(session: SparkSession): Unit = { + private def createTestTables(session: SparkSession): Unit = { import session.implicits._ - (1 to 100).map(i => (i, i.toString)).toDF("key", "value").createOrReplaceTempView("testdata") + (1 to 100).map(i => (i, i.toString)).toDF("key", "value") + .repartition(1) + .write + .format("parquet") + .saveAsTable("testdata") ((Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: (Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) .toDF("arraycol", "nestedarraycol") - .createOrReplaceTempView("arraydata") + .write + .format("parquet") + .saveAsTable("arraydata") (Tuple1(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: Tuple1(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: @@ -582,7 +592,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { Tuple1(Map(1 -> "a4", 2 -> "b4")) :: Tuple1(Map(1 -> "a5")) :: Nil) .toDF("mapcol") - .createOrReplaceTempView("mapdata") + .write + .format("parquet") + .saveAsTable("mapdata") session .read @@ -590,7 +602,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { .options(Map("delimiter" -> "\t", "header" -> "false")) .schema("a int, b float") .load(testFile("test-data/postgresql/agg.data")) - .createOrReplaceTempView("aggtest") + .write + .format("parquet") + .saveAsTable("aggtest") session .read @@ -616,7 +630,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { |string4 string """.stripMargin) .load(testFile("test-data/postgresql/onek.data")) - .createOrReplaceTempView("onek") + .write + .format("parquet") + .saveAsTable("onek") session .read @@ -642,7 +658,18 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { |string4 string """.stripMargin) .load(testFile("test-data/postgresql/tenk.data")) - .createOrReplaceTempView("tenk1") + .write + .format("parquet") + .saveAsTable("tenk1") + } + + private def removeTestTables(session: SparkSession): Unit = { + session.sql("DROP TABLE IF EXISTS testdata") + session.sql("DROP TABLE IF EXISTS arraydata") + session.sql("DROP TABLE IF EXISTS mapdata") + session.sql("DROP TABLE IF EXISTS aggtest") + session.sql("DROP TABLE IF EXISTS onek") + session.sql("DROP TABLE IF EXISTS tenk1") } private val originalTimeZone = TimeZone.getDefault @@ -650,20 +677,33 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { override def beforeAll(): Unit = { super.beforeAll() + createTestTables(spark) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting Locale.setDefault(Locale.US) RuleExecutor.resetMetrics() + CodeGenerator.resetCompileTime + WholeStageCodegenExec.resetCodeGenTime } override def afterAll(): Unit = { try { TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) + removeTestTables(spark) // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) + + val generateJavaTime = WholeStageCodegenExec.codeGenTime + val codegenInfo = + s""" + |=== Metrics of Whole-Stage Codegen === + |Total code generation time: ${generateJavaTime.toDouble / NANOS_PER_SECOND} seconds + |Total compile time: ${CodeGenerator.compileTime.toDouble / NANOS_PER_SECOND} seconds + """.stripMargin + logWarning(codegenInfo) } finally { super.afterAll() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 7e305e0504729..d51a4618af519 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -74,9 +74,14 @@ case class ComplexReflectData( mapFieldContainsNull: Map[Int, Option[Long]], dataField: Data) +case class InvalidInJava(`abstract`: Int) + class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession { import testImplicits._ + // To avoid syntax error thrown by genjavadoc, make this case class non-top level and private. + private case class InvalidInJava2(`0`: Int) + test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3), @@ -142,4 +147,20 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession Map(10 -> 100L, 20 -> 200L, 30 -> null), Row(null, "abc")))) } + + test("better error message when use java reserved keyword as field name") { + val e = intercept[UnsupportedOperationException] { + Seq(InvalidInJava(1)).toDS() + } + assert(e.getMessage.contains( + "`abstract` is not a valid identifier of Java and cannot be used as field name")) + } + + test("better error message when use invalid java identifier as field name") { + val e1 = intercept[UnsupportedOperationException] { + Seq(InvalidInJava2(1)).toDS() + } + assert(e1.getMessage.contains( + "`0` is not a valid identifier of Java and cannot be used as field name")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala index 04257642fac81..4e85f739b95a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala @@ -188,18 +188,26 @@ abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { if (result.length > 1) result(0) + result(1) else result.head } - protected def checkCreateTable(table: String): Unit = { - checkCreateTableOrView(TableIdentifier(table, Some("default")), "TABLE") + protected def checkCreateTable(table: String, serde: Boolean = false): Unit = { + checkCreateTableOrView(TableIdentifier(table, Some("default")), "TABLE", serde) } - protected def checkCreateView(table: String): Unit = { - checkCreateTableOrView(TableIdentifier(table, Some("default")), "VIEW") + protected def checkCreateView(table: String, serde: Boolean = false): Unit = { + checkCreateTableOrView(TableIdentifier(table, Some("default")), "VIEW", serde) } - private def checkCreateTableOrView(table: TableIdentifier, checkType: String): Unit = { + protected def checkCreateTableOrView( + table: TableIdentifier, + checkType: String, + serde: Boolean): Unit = { val db = table.database.getOrElse("default") val expected = spark.sharedState.externalCatalog.getTable(db, table.table) - val shownDDL = sql(s"SHOW CREATE TABLE ${table.quotedString}").head().getString(0) + val shownDDL = if (serde) { + sql(s"SHOW CREATE TABLE ${table.quotedString} AS SERDE").head().getString(0) + } else { + sql(s"SHOW CREATE TABLE ${table.quotedString}").head().getString(0) + } + sql(s"DROP $checkType ${table.quotedString}") try { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 99ea95089d71c..2d48f8df23e65 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -327,6 +327,9 @@ case class MyParser(spark: SparkSession, delegate: ParserInterface) extends Pars override def parseDataType(sqlText: String): DataType = delegate.parseDataType(sqlText) + + override def parseRawDataType(sqlText: String): DataType = + delegate.parseRawDataType(sqlText) } object MyExtensions { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 30b15a8710d8c..708b98e8fe15a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -481,7 +481,8 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared } } - DateTimeTestUtils.outstandingTimezones.foreach { timeZone => + DateTimeTestUtils.outstandingZoneIds.foreach { zid => + val timeZone = TimeZone.getTimeZone(zid) checkTimestampStats(DateType, DateTimeUtils.TimeZoneUTC, timeZone) { stats => assert(stats.min.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(start)) assert(stats.max.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(end - 1)) 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 ff8f94c68c5ee..754b24bef7cfa 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 @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort} import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, FileSourceScanExec, InputAdapter, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec} -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.datasources.FileScanRDD import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -288,10 +288,10 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark " or l.a in (select c from r where l.b < r.d)"), Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) - intercept[AnalysisException] { + checkAnswer( sql("select * from l where a not in (select c from r)" + - " or a not in (select c from r where c is not null)") - } + " or a not in (select c from r where c is not null)"), + Row(1, 2.0) :: Row(1, 2.0) :: Nil) } test("complex IN predicate subquery") { @@ -1357,11 +1357,9 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } - test("SPARK-27279: Reuse Subquery") { + test("SPARK-27279: Reuse Subquery", DisableAdaptiveExecution("reuse is dynamic in AQE")) { Seq(true, false).foreach { reuse => - withSQLConf(SQLConf.SUBQUERY_REUSE_ENABLED.key -> reuse.toString, - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - // when enable AQE, the reusedExchange is inserted when executed. + withSQLConf(SQLConf.SUBQUERY_REUSE_ENABLED.key -> reuse.toString) { val df = sql( """ |SELECT (SELECT avg(key) FROM testData) + (SELECT avg(key) FROM testData) 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 e0857ed6bc35a..08f41f6819a0e 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 @@ -33,7 +33,6 @@ import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.QueryExecutionListener - private case class FunctionResult(f1: String, f2: String) class UDFSuite extends QueryTest with SharedSparkSession { @@ -551,4 +550,47 @@ class UDFSuite extends QueryTest with SharedSparkSession { } assert(e.getMessage.contains("Invalid arguments for function cast")) } + + test("only one case class parameter") { + val f = (d: TestData) => d.key * d.value.toInt + val myUdf = udf(f) + val df = Seq(("data", TestData(50, "2"))).toDF("col1", "col2") + checkAnswer(df.select(myUdf(Column("col2"))), Row(100) :: Nil) + } + + test("one case class with primitive parameter") { + val f = (i: Int, p: TestData) => p.key * i + val myUdf = udf(f) + val df = Seq((2, TestData(50, "data"))).toDF("col1", "col2") + checkAnswer(df.select(myUdf(Column("col1"), Column("col2"))), Row(100) :: Nil) + } + + test("multiple case class parameters") { + val f = (d1: TestData, d2: TestData) => d1.key * d2.key + val myUdf = udf(f) + val df = Seq((TestData(10, "d1"), TestData(50, "d2"))).toDF("col1", "col2") + checkAnswer(df.select(myUdf(Column("col1"), Column("col2"))), Row(500) :: Nil) + } + + test("input case class parameter and return case class") { + val f = (d: TestData) => TestData(d.key * 2, "copy") + val myUdf = udf(f) + val df = Seq(("data", TestData(50, "d2"))).toDF("col1", "col2") + checkAnswer(df.select(myUdf(Column("col2"))), Row(Row(100, "copy")) :: Nil) + } + + test("any and case class parameter") { + val f = (any: Any, d: TestData) => s"${any.toString}, ${d.value}" + val myUdf = udf(f) + val df = Seq(("Hello", TestData(50, "World"))).toDF("col1", "col2") + checkAnswer(df.select(myUdf(Column("col1"), Column("col2"))), Row("Hello, World") :: Nil) + } + + test("nested case class parameter") { + val f = (y: Int, training: TrainingSales) => training.sales.year + y + val myUdf = udf(f) + val df = Seq((20, TrainingSales("training", CourseSales("course", 2000, 3.14)))) + .toDF("col1", "col2") + checkAnswer(df.select(myUdf(Column("col1"), Column("col2"))), Row(2020) :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index ba4200d84d46b..3244684c33965 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -257,6 +257,7 @@ class DataSourceV2SQLSuite } test("CreateTable: without USING clause") { + spark.conf.set(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED.key, "false") // unset this config to use the default v2 session catalog. spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) val testCatalog = catalog("testcat").asTableCatalog @@ -319,6 +320,37 @@ class DataSourceV2SQLSuite } } + test("CreateTableAsSelect: do not double execute on collect(), take() and other queries") { + val basicCatalog = catalog("testcat").asTableCatalog + val atomicCatalog = catalog("testcat_atomic").asTableCatalog + val basicIdentifier = "testcat.table_name" + val atomicIdentifier = "testcat_atomic.table_name" + + Seq((basicCatalog, basicIdentifier), (atomicCatalog, atomicIdentifier)).foreach { + case (catalog, identifier) => + val df = spark.sql(s"CREATE TABLE $identifier USING foo AS SELECT id, data FROM source") + + df.collect() + df.take(5) + df.tail(5) + df.where("true").collect() + df.where("true").take(5) + df.where("true").tail(5) + + val table = catalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == identifier) + assert(table.partitioning.isEmpty) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) + assert(table.schema == new StructType() + .add("id", LongType) + .add("data", StringType)) + + val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) + } + } + test("ReplaceTableAsSelect: basic v2 implementation.") { val basicCatalog = catalog("testcat").asTableCatalog val atomicCatalog = catalog("testcat_atomic").asTableCatalog @@ -346,6 +378,43 @@ class DataSourceV2SQLSuite } } + Seq("REPLACE", "CREATE OR REPLACE").foreach { cmd => + test(s"ReplaceTableAsSelect: do not double execute $cmd on collect()") { + val basicCatalog = catalog("testcat").asTableCatalog + val atomicCatalog = catalog("testcat_atomic").asTableCatalog + val basicIdentifier = "testcat.table_name" + val atomicIdentifier = "testcat_atomic.table_name" + + Seq((basicCatalog, basicIdentifier), (atomicCatalog, atomicIdentifier)).foreach { + case (catalog, identifier) => + spark.sql(s"CREATE TABLE $identifier USING foo AS SELECT id, data FROM source") + val originalTable = catalog.loadTable(Identifier.of(Array(), "table_name")) + + val df = spark.sql(s"$cmd TABLE $identifier USING foo AS SELECT id FROM source") + + df.collect() + df.take(5) + df.tail(5) + df.where("true").collect() + df.where("true").take(5) + df.where("true").tail(5) + + val replacedTable = catalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(replacedTable != originalTable, "Table should have been replaced.") + assert(replacedTable.name == identifier) + assert(replacedTable.partitioning.isEmpty) + assert(replacedTable.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) + assert(replacedTable.schema == new StructType().add("id", LongType)) + + val rdd = spark.sparkContext.parallelize(replacedTable.asInstanceOf[InMemoryTable].rows) + checkAnswer( + spark.internalCreateDataFrame(rdd, replacedTable.schema), + spark.table("source").select("id")) + } + } + } + test("ReplaceTableAsSelect: Non-atomic catalog drops the table if the write fails.") { spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") val testCatalog = catalog("testcat").asTableCatalog @@ -613,6 +682,7 @@ class DataSourceV2SQLSuite } test("CreateTableAsSelect: without USING clause") { + spark.conf.set(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED.key, "false") // unset this config to use the default v2 session catalog. spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) val testCatalog = catalog("testcat").asTableCatalog @@ -839,6 +909,23 @@ class DataSourceV2SQLSuite assert(exception.getMessage.contains("The database name is not valid: a.b")) } + test("ShowViews: using v1 catalog, db name with multipartIdentifier ('a.b') is not allowed.") { + val exception = intercept[AnalysisException] { + sql("SHOW TABLES FROM a.b") + } + + assert(exception.getMessage.contains("The database name is not valid: a.b")) + } + + test("ShowViews: using v2 catalog, command not supported.") { + val exception = intercept[AnalysisException] { + sql("SHOW VIEWS FROM testcat") + } + + assert(exception.getMessage.contains("Catalog testcat doesn't support SHOW VIEWS," + + " only SessionCatalog supports this command.")) + } + test("ShowTables: using v2 catalog with empty namespace") { spark.sql("CREATE TABLE testcat.table (id bigint, data string) USING foo") runShowTablesSql("SHOW TABLES FROM testcat", Seq(Row("", "table"))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala index 0fd6cf1b6746c..b88ad5218fcd2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala @@ -463,5 +463,22 @@ trait InsertIntoSQLOnlyTests } } } + + test("do not double insert on INSERT INTO collect()") { + val t1 = s"${catalogAndNamespace}tbl" + withTableAndData(t1) { view => + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") + val df = sql(s"INSERT INTO TABLE $t1 SELECT * FROM $view") + + df.collect() + df.take(5) + df.tail(5) + df.where("true").collect() + df.where("true").take(5) + df.where("true").tail(5) + + verifyTable(t1, spark.table(view)) + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala index 9e77f618eded6..22c5b651f7e12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala @@ -23,7 +23,6 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ import org.apache.spark.sql.execution.adaptive._ -import org.apache.spark.sql.execution.adaptive.CoalesceShufflePartitions.COALESCED_SHUFFLE_READER_DESCRIPTION import org.apache.spark.sql.execution.adaptive.CustomShuffleReaderExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.functions._ @@ -108,7 +107,7 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl val finalPlan = agg.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan val shuffleReaders = finalPlan.collect { - case r @ CustomShuffleReaderExec(_, _, COALESCED_SHUFFLE_READER_DESCRIPTION) => r + case r @ CoalescedShuffleReader() => r } assert(shuffleReaders.length === 1) minNumPostShufflePartitions match { @@ -155,7 +154,7 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan val shuffleReaders = finalPlan.collect { - case r @ CustomShuffleReaderExec(_, _, COALESCED_SHUFFLE_READER_DESCRIPTION) => r + case r @ CoalescedShuffleReader() => r } assert(shuffleReaders.length === 2) minNumPostShufflePartitions match { @@ -207,7 +206,7 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan val shuffleReaders = finalPlan.collect { - case r @ CustomShuffleReaderExec(_, _, COALESCED_SHUFFLE_READER_DESCRIPTION) => r + case r @ CoalescedShuffleReader() => r } assert(shuffleReaders.length === 2) minNumPostShufflePartitions match { @@ -259,7 +258,7 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan val shuffleReaders = finalPlan.collect { - case r @ CustomShuffleReaderExec(_, _, COALESCED_SHUFFLE_READER_DESCRIPTION) => r + case r @ CoalescedShuffleReader() => r } assert(shuffleReaders.length === 2) minNumPostShufflePartitions match { @@ -302,7 +301,7 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan val shuffleReaders = finalPlan.collect { - case r @ CustomShuffleReaderExec(_, _, COALESCED_SHUFFLE_READER_DESCRIPTION) => r + case r @ CoalescedShuffleReader() => r } assert(shuffleReaders.length === 0) } finally { @@ -332,7 +331,7 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl }.length == 2) assert( finalPlan.collect { - case p @ CustomShuffleReaderExec(_, _, COALESCED_SHUFFLE_READER_DESCRIPTION) => p + case r @ CoalescedShuffleReader() => r }.length == 3) @@ -383,7 +382,7 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl .asInstanceOf[AdaptiveSparkPlanExec].executedPlan assert( finalPlan.collect { - case p @ CustomShuffleReaderExec(_, _, COALESCED_SHUFFLE_READER_DESCRIPTION) => p + case r @ CoalescedShuffleReader() => r }.isEmpty) } withSparkSession(test, 200, None) @@ -404,9 +403,15 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl // the shuffle partition numbers. assert( finalPlan.collect { - case p @ CustomShuffleReaderExec(_, _, COALESCED_SHUFFLE_READER_DESCRIPTION) => p + case r @ CoalescedShuffleReader() => r }.isEmpty) } withSparkSession(test, 100, None) } } + +object CoalescedShuffleReader { + def unapply(reader: CustomShuffleReaderExec): Boolean = { + !reader.isLocalReader && !reader.hasSkewedPartition && reader.hasCoalescedPartition + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala index 1e90754ad7721..b27a940c364a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala @@ -18,30 +18,27 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.expressions.scalalang.typed -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +// Disable AQE because the WholeStageCodegenExec is added when running QueryStageExec @deprecated("This test suite will be removed.", "3.0.0") class DeprecatedWholeStageCodegenSuite extends QueryTest with SharedSparkSession - with AdaptiveSparkPlanHelper { + with DisableAdaptiveExecutionSuite { test("simple typed UDAF should be included in WholeStageCodegen") { - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - // With enable AQE, the WholeStageCodegenExec rule is applied when running QueryStageExec. - import testImplicits._ + import testImplicits._ - val ds = Seq(("a", 10), ("b", 1), ("b", 2), ("c", 1)).toDS() - .groupByKey(_._1).agg(typed.sum(_._2)) + val ds = Seq(("a", 10), ("b", 1), ("b", 2), ("c", 1)).toDS() + .groupByKey(_._1).agg(typed.sum(_._2)) - val plan = ds.queryExecution.executedPlan - assert(find(plan)(p => - p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) - assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) - } + val plan = ds.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) + assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala index 7fbfa73623c85..28e82aa14e0d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalog.Table import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, HintInfo, Join, JoinHint} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType @@ -170,4 +171,25 @@ class GlobalTempViewSuite extends QueryTest with SharedSparkSession { isTemporary = true).toString) } } + + test("broadcast hint on global temp view") { + withGlobalTempView("v1") { + spark.range(10).createGlobalTempView("v1") + withTempView("v2") { + spark.range(10).createTempView("v2") + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + Seq( + "SELECT /*+ MAPJOIN(v1) */ * FROM global_temp.v1, v2 WHERE v1.id = v2.id", + "SELECT /*+ MAPJOIN(global_temp.v1) */ * FROM global_temp.v1, v2 WHERE v1.id = v2.id" + ).foreach { statement => + sql(statement).queryExecution.optimizedPlan match { + case Join(_, _, _, _, JoinHint(Some(HintInfo(Some(BROADCAST))), None)) => + case _ => fail("broadcast hint not found in a left-side table") + } + } + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala index bf7cbaad7384b..5e81c74420fd0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.sql.connector.InMemoryTableCatalog import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} class HiveResultSuite extends SharedSparkSession { @@ -68,4 +69,35 @@ class HiveResultSuite extends SharedSparkSession { val result = HiveResult.hiveResultString(executedPlan) assert(result.head === "0.00000000") } + + test("SHOW TABLES in hive result") { + withSQLConf("spark.sql.catalog.testcat" -> classOf[InMemoryTableCatalog].getName) { + Seq(("testcat.ns", "tbl", "foo"), ("spark_catalog.default", "tbl", "csv")).foreach { + case (ns, tbl, source) => + withTable(s"$ns.$tbl") { + spark.sql(s"CREATE TABLE $ns.$tbl (id bigint) USING $source") + val df = spark.sql(s"SHOW TABLES FROM $ns") + val executedPlan = df.queryExecution.executedPlan + assert(HiveResult.hiveResultString(executedPlan).head == tbl) + } + } + } + } + + test("DESCRIBE TABLE in hive result") { + withSQLConf("spark.sql.catalog.testcat" -> classOf[InMemoryTableCatalog].getName) { + Seq(("testcat.ns", "tbl", "foo"), ("spark_catalog.default", "tbl", "csv")).foreach { + case (ns, tbl, source) => + withTable(s"$ns.$tbl") { + spark.sql(s"CREATE TABLE $ns.$tbl (id bigint COMMENT 'col1') USING $source") + val df = spark.sql(s"DESCRIBE $ns.$tbl") + val executedPlan = df.queryExecution.executedPlan + val expected = "id " + + "\tbigint " + + "\tcol1 " + assert(HiveResult.hiveResultString(executedPlan).head == expected) + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala index 311f84c07a955..5bcec9b1e517c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala @@ -22,6 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.TPCDSQuerySuite import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Final} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Generate, Join, LocalRelation, LogicalPlan, Range, Sample, Union, Window} +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -29,22 +30,9 @@ import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.window.WindowExec -import org.apache.spark.sql.internal.SQLConf -class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite { - - var originalValue: String = _ - // when enable AQE, the 'AdaptiveSparkPlanExec' node does not have a logical plan link - override def beforeAll(): Unit = { - super.beforeAll() - originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") - } - - override def afterAll(): Unit = { - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) - super.afterAll() - } +// Disable AQE because AdaptiveSparkPlanExec does not have a logical plan link +class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite with DisableAdaptiveExecutionSuite { override protected def checkGeneratedCode( plan: SparkPlan, checkMethodCodeSize: Boolean = true): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index dfa8046cd6222..d428b7ebc0e91 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range, Repartition, Sort, Union} import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchangeExec, ReuseExchange, ShuffleExchangeExec} @@ -752,7 +752,8 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { } test("SPARK-24556: always rewrite output partitioning in ReusedExchangeExec " + - "and InMemoryTableScanExec") { + "and InMemoryTableScanExec", + DisableAdaptiveExecution("Reuse is dynamic in AQE")) { def checkOutputPartitioningRewrite( plans: Seq[SparkPlan], expectedPartitioningClass: Class[_]): Unit = { @@ -782,8 +783,7 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { checkOutputPartitioningRewrite(inMemoryScan, expectedPartitioningClass) } // when enable AQE, the reusedExchange is inserted when executed. - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { // ReusedExchange is HashPartitioning val df1 = Seq(1 -> "a").toDF("i", "j").repartition($"i") val df2 = Seq(1 -> "a").toDF("i", "j").repartition($"i") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala index 2cd4c98716c28..7acc33c43b19d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala @@ -33,8 +33,6 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite { } val estimatedPartitionStartIndices = ShufflePartitionsUtil.coalescePartitions( mapOutputStatistics, - 0, - bytesByPartitionIdArray.head.length, targetSize, minNumPartitions) assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 06a016fac5300..f7396ee2a89c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec @@ -28,23 +29,12 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructType} -class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { +// Disable AQE because the WholeStageCodegenExec is added when running QueryStageExec +class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + with DisableAdaptiveExecutionSuite { import testImplicits._ - var originalValue: String = _ - // With on AQE, the WholeStageCodegenExec is added when running QueryStageExec. - override def beforeAll(): Unit = { - super.beforeAll() - originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") - } - - override def afterAll(): Unit = { - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) - super.afterAll() - } - test("range/filter should be combined") { val df = spark.range(10).filter("id = 1").selectExpr("id + 1") val plan = df.queryExecution.executedPlan diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index fcca23d7751f0..0ba1f9ea5d28a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -23,14 +23,15 @@ import java.net.URI import org.apache.log4j.Level import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.execution.{ReusedSubqueryExec, ShuffledRowRDD, SparkPlan} -import org.apache.spark.sql.execution.adaptive.OptimizeLocalShuffleReader.LOCAL_SHUFFLE_READER_DESCRIPTION +import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec} -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildRight, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, SortMergeJoinExec} import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.util.Utils class AdaptiveQueryExecSuite @@ -94,14 +95,14 @@ class AdaptiveQueryExecSuite } private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = { - collectInPlanAndSubqueries(plan) { + collectWithSubqueries(plan) { case ShuffleQueryStageExec(_, e: ReusedExchangeExec) => e case BroadcastQueryStageExec(_, e: ReusedExchangeExec) => e } } private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { - collectInPlanAndSubqueries(plan) { + collectWithSubqueries(plan) { case e: ReusedSubqueryExec => e } } @@ -113,7 +114,7 @@ class AdaptiveQueryExecSuite }.length val numLocalReaders = collect(plan) { - case reader @ CustomShuffleReaderExec(_, _, LOCAL_SHUFFLE_READER_DESCRIPTION) => reader + case reader: CustomShuffleReaderExec if reader.isLocalReader => reader } numLocalReaders.foreach { r => val rdd = r.execute() @@ -149,7 +150,7 @@ class AdaptiveQueryExecSuite val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) val localReaders = collect(adaptivePlan) { - case reader @ CustomShuffleReaderExec(_, _, LOCAL_SHUFFLE_READER_DESCRIPTION) => reader + case reader: CustomShuffleReaderExec if reader.isLocalReader => reader } assert(localReaders.length == 2) val localShuffleRDD0 = localReaders(0).execute().asInstanceOf[ShuffledRowRDD] @@ -181,7 +182,7 @@ class AdaptiveQueryExecSuite val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) val localReaders = collect(adaptivePlan) { - case reader @ CustomShuffleReaderExec(_, _, LOCAL_SHUFFLE_READER_DESCRIPTION) => reader + case reader: CustomShuffleReaderExec if reader.isLocalReader => reader } assert(localReaders.length == 2) val localShuffleRDD0 = localReaders(0).execute().asInstanceOf[ShuffledRowRDD] @@ -615,6 +616,7 @@ class AdaptiveQueryExecSuite withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "2000", SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "2000") { withTempView("skewData1", "skewData2") { spark @@ -780,5 +782,131 @@ class AdaptiveQueryExecSuite } } } -} + test("SPARK-31384: avoid NPE in OptimizeSkewedJoin when there's 0 partition plan") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTempView("t2") { + // create DataFrame with 0 partition + spark.createDataFrame(sparkContext.emptyRDD[Row], new StructType().add("b", IntegerType)) + .createOrReplaceTempView("t2") + // should run successfully without NPE + runAdaptiveAndVerifyResult("SELECT * FROM testData2 t1 left semi join t2 ON t1.a=t2.b") + } + } + } + + test("metrics of the shuffle reader") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT key FROM testData GROUP BY key") + val readers = collect(adaptivePlan) { + case r: CustomShuffleReaderExec => r + } + assert(readers.length == 1) + val reader = readers.head + assert(!reader.isLocalReader) + assert(!reader.hasSkewedPartition) + assert(reader.hasCoalescedPartition) + assert(reader.metrics.keys.toSeq.sorted == Seq( + "avgPartitionDataSize", "maxPartitionDataSize", "minPartitionDataSize", "numPartitions")) + assert(reader.metrics("numPartitions").value == reader.partitionSpecs.length) + assert(reader.metrics("avgPartitionDataSize").value > 0) + assert(reader.metrics("maxPartitionDataSize").value > 0) + assert(reader.metrics("minPartitionDataSize").value > 0) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val join = collect(adaptivePlan) { + case j: BroadcastHashJoinExec => j + }.head + assert(join.buildSide == BuildLeft) + + val readers = collect(join.right) { + case r: CustomShuffleReaderExec => r + } + assert(readers.length == 1) + val reader = readers.head + assert(reader.isLocalReader) + assert(reader.metrics.keys.toSeq == Seq("numPartitions")) + assert(reader.metrics("numPartitions").value == reader.partitionSpecs.length) + } + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "2000", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "2000") { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 2 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 join skewData2 ON key1 = key2") + val reader = collect(adaptivePlan) { + case r: CustomShuffleReaderExec => r + }.head + assert(!reader.isLocalReader) + assert(reader.hasSkewedPartition) + assert(reader.hasCoalescedPartition) + assert(reader.metrics.contains("numSkewedPartitions")) + assert(reader.metrics("numSkewedPartitions").value > 0) + } + } + } + } + + test("control a plan explain mode in listeners via SQLConf") { + + def checkPlanDescription(mode: String, expected: Seq[String]): Unit = { + var checkDone = false + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case SparkListenerSQLAdaptiveExecutionUpdate(_, planDescription, _) => + assert(expected.forall(planDescription.contains)) + checkDone = true + case _ => // ignore other events + } + } + } + spark.sparkContext.addSparkListener(listener) + withSQLConf(SQLConf.UI_EXPLAIN_MODE.key -> mode, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val dfApdaptive = sql("SELECT * FROM testData JOIN testData2 ON key = a WHERE value = '1'") + try { + checkAnswer(dfApdaptive, Row(1, "1", 1, 1) :: Row(1, "1", 1, 2) :: Nil) + spark.sparkContext.listenerBus.waitUntilEmpty() + assert(checkDone) + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } + } + + Seq(("simple", Seq("== Physical Plan ==")), + ("extended", Seq("== Parsed Logical Plan ==", "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==", "== Physical Plan ==")), + ("codegen", Seq("WholeStageCodegen subtrees")), + ("cost", Seq("== Optimized Logical Plan ==", "Statistics(sizeInBytes")), + ("formatted", Seq("== Physical Plan ==", "Output", "Arguments"))).foreach { + case (mode, expected) => + checkPlanDescription(mode, expected) + } + } + + test("SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of write commands") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + val plan = sql("CREATE TABLE t1 AS SELECT 1 col").queryExecution.executedPlan + assert(plan.isInstanceOf[DataWritingCommandExec]) + assert(plan.asInstanceOf[DataWritingCommandExec].child.isInstanceOf[AdaptiveSparkPlanExec]) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveTestUtils.scala new file mode 100644 index 0000000000000..ddaeb57d31547 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveTestUtils.scala @@ -0,0 +1,93 @@ +/* + * 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.adaptive + +import java.io.{PrintWriter, StringWriter} + +import org.scalactic.source.Position +import org.scalatest.Tag + +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils + +/** + * Test with this tag will be ignored if the test suite extends `EnableAdaptiveExecutionSuite`. + * Otherwise, it will be executed with adaptive execution disabled. + */ +case class DisableAdaptiveExecution(reason: String) extends Tag("DisableAdaptiveExecution") + +/** + * Helper trait that enables AQE for all tests regardless of default config values, except that + * tests tagged with [[DisableAdaptiveExecution]] will be skipped. + */ +trait EnableAdaptiveExecutionSuite extends SQLTestUtils { + protected val forceApply = true + + override protected def test(testName: String, testTags: Tag*)(testFun: => Any) + (implicit pos: Position): Unit = { + if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { + // we ignore the test here but assume that another test suite which extends + // `DisableAdaptiveExecutionSuite` will test it anyway to ensure test coverage + ignore(testName + " (disabled when AQE is on)", testTags: _*)(testFun) + } else { + super.test(testName, testTags: _*) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> forceApply.toString) { + testFun + } + } + } + } +} + +/** + * Helper trait that disables AQE for all tests regardless of default config values. + */ +trait DisableAdaptiveExecutionSuite extends SQLTestUtils { + override protected def test(testName: String, testTags: Tag*)(testFun: => Any) + (implicit pos: Position): Unit = { + super.test(testName, testTags: _*) { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + testFun + } + } + } +} + +object AdaptiveTestUtils { + def assertExceptionMessage(e: Exception, expected: String): Unit = { + val stringWriter = new StringWriter() + e.printStackTrace(new PrintWriter(stringWriter)) + val errorMsg = stringWriter.toString + assert(errorMsg.contains(expected)) + } + + def assertExceptionCause(t: Throwable, causeClass: Class[_]): Unit = { + var c = t.getCause + var foundCause = false + while (c != null && !foundCause) { + if (causeClass.isAssignableFrom(c.getClass)) { + foundCause = true + } else { + c = c.getCause + } + } + assert(foundCause, s"Can not find cause: $causeClass") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala index fdb23d5be78a1..1e6e59456c887 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala @@ -1208,15 +1208,13 @@ class ArrowConvertersSuite extends SharedSparkSession { spark.conf.unset(SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key) } - testQuietly("unsupported types") { - def runUnsupported(block: => Unit): Unit = { - val msg = intercept[UnsupportedOperationException] { - block - } - assert(msg.getMessage.contains("is not supported")) + testQuietly("interval is unsupported for arrow") { + val e = intercept[SparkException] { + calenderIntervalData.toDF().toArrowBatchRdd.collect() } - runUnsupported { calenderIntervalData.toDF().toArrowBatchRdd.collect() } + assert(e.getCause.isInstanceOf[UnsupportedOperationException]) + assert(e.getCause.getMessage.contains("Unsupported data type: interval")) } test("test Arrow Validator") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala index 8de9b8d076e0f..5fa5b9b06a60b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.benchmark import java.sql.Timestamp import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, LA} +import org.apache.spark.sql.internal.SQLConf /** * Synthetic benchmark for date and timestamp functions. @@ -53,96 +55,105 @@ object DateTimeBenchmark extends SqlBasedBenchmark { } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - val N = 10000000 - runBenchmark("Extract components") { - run(N, "cast to timestamp", "cast(id as timestamp)") - run(N, "year") - run(N, "quarter") - run(N, "month") - run(N, "weekofyear") - run(N, "day") - run(N, "dayofyear") - run(N, "dayofmonth") - run(N, "dayofweek") - run(N, "weekday") - run(N, "hour") - run(N, "minute") - run(N, "second") - } - runBenchmark("Current date and time") { - run(N, "current_date", "current_date") - run(N, "current_timestamp", "current_timestamp") - } - runBenchmark("Date arithmetic") { - val dateExpr = "cast(cast(id as timestamp) as date)" - run(N, "cast to date", dateExpr) - run(N, "last_day", s"last_day($dateExpr)") - run(N, "next_day", s"next_day($dateExpr, 'TU')") - run(N, "date_add", s"date_add($dateExpr, 10)") - run(N, "date_sub", s"date_sub($dateExpr, 10)") - run(N, "add_months", s"add_months($dateExpr, 10)") - } - runBenchmark("Formatting dates") { - val dateExpr = "cast(cast(id as timestamp) as date)" - run(N, "format date", s"date_format($dateExpr, 'MMM yyyy')") - } - runBenchmark("Formatting timestamps") { - run(N, "from_unixtime", "from_unixtime(id, 'yyyy-MM-dd HH:mm:ss.SSSSSS')") - } - runBenchmark("Convert timestamps") { - val timestampExpr = "cast(id as timestamp)" - run(N, "from_utc_timestamp", s"from_utc_timestamp($timestampExpr, 'CET')") - run(N, "to_utc_timestamp", s"to_utc_timestamp($timestampExpr, 'CET')") - } - runBenchmark("Intervals") { - val (start, end) = ("cast(id as timestamp)", "cast((id+8640000) as timestamp)") - run(N, "cast interval", start, end) - run(N, "datediff", s"datediff($start, $end)") - run(N, "months_between", s"months_between($start, $end)") - run(1000000, "window", s"window($start, 100, 10, 1)") - } - runBenchmark("Truncation") { - val timestampExpr = "cast(id as timestamp)" - Seq("YEAR", "YYYY", "YY", "MON", "MONTH", "MM", "DAY", "DD", "HOUR", "MINUTE", - "SECOND", "WEEK", "QUARTER").foreach { level => - run(N, s"date_trunc $level", s"date_trunc('$level', $timestampExpr)") - } - val dateExpr = "cast(cast(id as timestamp) as date)" - Seq("year", "yyyy", "yy", "mon", "month", "mm").foreach { level => - run(N, s"trunc $level", s"trunc('$level', $dateExpr)") - } - } - runBenchmark("Parsing") { - val n = 1000000 - val timestampStrExpr = "concat('2019-01-27 11:02:01.', rpad(mod(id, 1000), 3, '0'))" - val pattern = "'yyyy-MM-dd HH:mm:ss.SSS'" - run(n, "to timestamp str", timestampStrExpr) - run(n, "to_timestamp", s"to_timestamp($timestampStrExpr, $pattern)") - run(n, "to_unix_timestamp", s"to_unix_timestamp($timestampStrExpr, $pattern)") - val dateStrExpr = "concat('2019-01-', lpad(mod(id, 25), 2, '0'))" - run(n, "to date str", dateStrExpr) - run(n, "to_date", s"to_date($dateStrExpr, 'yyyy-MM-dd')") - } - runBenchmark("Conversion from/to external types") { - import spark.implicits._ - val rowsNum = 5000000 - val numIters = 3 - val benchmark = new Benchmark("To/from java.sql.Timestamp", rowsNum, output = output) - benchmark.addCase("From java.sql.Timestamp", numIters) { _ => - spark.range(rowsNum) - .map(millis => new Timestamp(millis)) - .noop() - } - benchmark.addCase("Collect longs", numIters) { _ => - spark.range(0, rowsNum, 1, 1) - .collect() - } - benchmark.addCase("Collect timestamps", numIters) { _ => - spark.range(0, rowsNum, 1, 1) - .map(millis => new Timestamp(millis)) - .collect() + withDefaultTimeZone(LA) { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> LA.getId) { + val N = 10000000 + runBenchmark("Extract components") { + run(N, "cast to timestamp", "cast(id as timestamp)") + run(N, "year") + run(N, "quarter") + run(N, "month") + run(N, "weekofyear") + run(N, "day") + run(N, "dayofyear") + run(N, "dayofmonth") + run(N, "dayofweek") + run(N, "weekday") + run(N, "hour") + run(N, "minute") + run(N, "second") + } + runBenchmark("Current date and time") { + run(N, "current_date", "current_date") + run(N, "current_timestamp", "current_timestamp") + } + runBenchmark("Date arithmetic") { + val dateExpr = "cast(cast(id as timestamp) as date)" + run(N, "cast to date", dateExpr) + run(N, "last_day", s"last_day($dateExpr)") + run(N, "next_day", s"next_day($dateExpr, 'TU')") + run(N, "date_add", s"date_add($dateExpr, 10)") + run(N, "date_sub", s"date_sub($dateExpr, 10)") + run(N, "add_months", s"add_months($dateExpr, 10)") + } + runBenchmark("Formatting dates") { + val dateExpr = "cast(cast(id as timestamp) as date)" + run(N, "format date", s"date_format($dateExpr, 'MMM yyyy')") + } + runBenchmark("Formatting timestamps") { + run(N, "from_unixtime", "from_unixtime(id, 'yyyy-MM-dd HH:mm:ss.SSSSSS')") + } + runBenchmark("Convert timestamps") { + val timestampExpr = "cast(id as timestamp)" + run(N, "from_utc_timestamp", s"from_utc_timestamp($timestampExpr, 'CET')") + run(N, "to_utc_timestamp", s"to_utc_timestamp($timestampExpr, 'CET')") + } + runBenchmark("Intervals") { + val (start, end) = ("cast(id as timestamp)", "cast((id+8640000) as timestamp)") + run(N, "cast interval", start, end) + run(N, "datediff", s"datediff($start, $end)") + run(N, "months_between", s"months_between($start, $end)") + run(1000000, "window", s"window($start, 100, 10, 1)") + } + runBenchmark("Truncation") { + val timestampExpr = "cast(id as timestamp)" + Seq("YEAR", "YYYY", "YY", "MON", "MONTH", "MM", "DAY", "DD", "HOUR", "MINUTE", + "SECOND", "WEEK", "QUARTER").foreach { level => + run(N, s"date_trunc $level", s"date_trunc('$level', $timestampExpr)") + } + val dateExpr = "cast(cast(id as timestamp) as date)" + Seq("year", "yyyy", "yy", "mon", "month", "mm").foreach { level => + run(N, s"trunc $level", s"trunc('$level', $dateExpr)") + } + } + runBenchmark("Parsing") { + val n = 1000000 + val timestampStrExpr = "concat('2019-01-27 11:02:01.', cast(mod(id, 1000) as string))" + val pattern = "'yyyy-MM-dd HH:mm:ss.SSS'" + run(n, "to timestamp str", timestampStrExpr) + run(n, "to_timestamp", s"to_timestamp($timestampStrExpr, $pattern)") + run(n, "to_unix_timestamp", s"to_unix_timestamp($timestampStrExpr, $pattern)") + val dateStrExpr = "concat('2019-01-', lpad(mod(id, 25), 2, '0'))" + run(n, "to date str", dateStrExpr) + run(n, "to_date", s"to_date($dateStrExpr, 'yyyy-MM-dd')") + } + runBenchmark("Conversion from/to external types") { + import spark.implicits._ + val rowsNum = 5000000 + val numIters = 3 + val benchmark = new Benchmark("To/from Java's date-time", rowsNum, output = output) + benchmark.addCase("From java.sql.Date", numIters) { _ => + spark.range(rowsNum) + .map(millis => new java.sql.Date(millis)) + .noop() + } + benchmark.addCase("From java.sql.Timestamp", numIters) { _ => + spark.range(rowsNum) + .map(millis => new Timestamp(millis)) + .noop() + } + benchmark.addCase("Collect longs", numIters) { _ => + spark.range(0, rowsNum, 1, 1) + .collect() + } + benchmark.addCase("Collect timestamps", numIters) { _ => + spark.range(0, rowsNum, 1, 1) + .map(millis => new Timestamp(millis)) + .collect() + } + benchmark.run() + } } - benchmark.run() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala new file mode 100644 index 0000000000000..077ac28c149ee --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala @@ -0,0 +1,221 @@ +/* + * 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.benchmark + +import java.io.File +import java.time.{LocalDate, LocalDateTime, LocalTime, ZoneOffset} + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.util.DateTimeConstants.SECONDS_PER_DAY +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, LA} +import org.apache.spark.sql.internal.SQLConf + +/** + * Synthetic benchmark for rebasing of date and timestamp in read/write. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "sql/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/DateTimeRebaseBenchmark-results.txt". + * }}} + */ +object DateTimeRebaseBenchmark extends SqlBasedBenchmark { + import spark.implicits._ + + private def genTs(cardinality: Int, start: LocalDateTime, end: LocalDateTime): DataFrame = { + val startSec = start.toEpochSecond(ZoneOffset.UTC) + val endSec = end.toEpochSecond(ZoneOffset.UTC) + spark.range(0, cardinality, 1, 1) + .select((($"id" % (endSec - startSec)) + startSec).as("seconds")) + .select($"seconds".cast("timestamp").as("ts")) + } + + private def genTsAfter1582(cardinality: Int): DataFrame = { + val start = LocalDateTime.of(1582, 10, 15, 0, 0, 0) + val end = LocalDateTime.of(3000, 1, 1, 0, 0, 0) + genTs(cardinality, start, end) + } + + private def genTsBefore1582(cardinality: Int): DataFrame = { + val start = LocalDateTime.of(10, 1, 1, 0, 0, 0) + val end = LocalDateTime.of(1580, 1, 1, 0, 0, 0) + genTs(cardinality, start, end) + } + + private def genDate(cardinality: Int, start: LocalDate, end: LocalDate): DataFrame = { + val startSec = LocalDateTime.of(start, LocalTime.MIDNIGHT).toEpochSecond(ZoneOffset.UTC) + val endSec = LocalDateTime.of(end, LocalTime.MIDNIGHT).toEpochSecond(ZoneOffset.UTC) + spark.range(0, cardinality * SECONDS_PER_DAY, SECONDS_PER_DAY, 1) + .select((($"id" % (endSec - startSec)) + startSec).as("seconds")) + .select($"seconds".cast("timestamp").as("ts")) + .select($"ts".cast("date").as("date")) + } + + private def genDateAfter1582(cardinality: Int): DataFrame = { + val start = LocalDate.of(1582, 10, 15) + val end = LocalDate.of(3000, 1, 1) + genDate(cardinality, start, end) + } + + private def genDateBefore1582(cardinality: Int): DataFrame = { + val start = LocalDate.of(10, 1, 1) + val end = LocalDate.of(1580, 1, 1) + genDate(cardinality, start, end) + } + + private def genDF(cardinality: Int, dateTime: String, after1582: Boolean): DataFrame = { + (dateTime, after1582) match { + case ("date", true) => genDateAfter1582(cardinality) + case ("date", false) => genDateBefore1582(cardinality) + case ("timestamp", true) => genTsAfter1582(cardinality) + case ("timestamp", false) => genTsBefore1582(cardinality) + case _ => throw new IllegalArgumentException( + s"cardinality = $cardinality dateTime = $dateTime after1582 = $after1582") + } + } + + private def benchmarkInputs(benchmark: Benchmark, rowsNum: Int, dateTime: String): Unit = { + benchmark.addCase("after 1582, noop", 1) { _ => + genDF(rowsNum, dateTime, after1582 = true).noop() + } + benchmark.addCase("before 1582, noop", 1) { _ => + genDF(rowsNum, dateTime, after1582 = false).noop() + } + } + + private def flagToStr(flag: Boolean): String = { + if (flag) "on" else "off" + } + + private def caseName( + after1582: Boolean, + rebase: Option[Boolean] = None, + vec: Option[Boolean] = None): String = { + val period = if (after1582) "after" else "before" + val vecFlag = vec.map(flagToStr).map(flag => s", vec $flag").getOrElse("") + val rebaseFlag = rebase.map(flagToStr).map(flag => s", rebase $flag").getOrElse("") + s"$period 1582$vecFlag$rebaseFlag" + } + + private def getPath( + basePath: File, + dateTime: String, + after1582: Boolean, + rebase: Option[Boolean] = None): String = { + val period = if (after1582) "after" else "before" + val rebaseFlag = rebase.map(flagToStr).map(flag => s"_$flag").getOrElse("") + basePath.getAbsolutePath + s"/${dateTime}_${period}_1582$rebaseFlag" + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val rowsNum = 100000000 + + withDefaultTimeZone(LA) { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> LA.getId) { + withTempPath { path => + runBenchmark("Rebasing dates/timestamps in Parquet datasource") { + Seq("date", "timestamp").foreach { dateTime => + val benchmark = new Benchmark( + s"Save ${dateTime}s to parquet", + rowsNum, + output = output) + benchmarkInputs(benchmark, rowsNum, dateTime) + Seq(true, false).foreach { after1582 => + Seq(false, true).foreach { rebase => + benchmark.addCase(caseName(after1582, Some(rebase)), 1) { _ => + withSQLConf( + SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_WRITE.key -> rebase.toString) { + genDF(rowsNum, dateTime, after1582) + .write + .mode("overwrite") + .format("parquet") + .save(getPath(path, dateTime, after1582, Some(rebase))) + } + } + } + } + benchmark.run() + + val benchmark2 = new Benchmark( + s"Load ${dateTime}s from parquet", rowsNum, output = output) + Seq(true, false).foreach { after1582 => + Seq(false, true).foreach { vec => + Seq(false, true).foreach { rebase => + benchmark2.addCase(caseName(after1582, Some(rebase), Some(vec)), 3) { _ => + withSQLConf( + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vec.toString, + SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_READ.key -> rebase.toString) { + spark.read + .format("parquet") + .load(getPath(path, dateTime, after1582, Some(rebase))) + .noop() + } + } + } + } + } + benchmark2.run() + } + } + } + + withTempPath { path => + runBenchmark("Rebasing dates/timestamps in ORC datasource") { + Seq("date", "timestamp").foreach { dateTime => + val benchmark = new Benchmark(s"Save ${dateTime}s to ORC", rowsNum, output = output) + benchmarkInputs(benchmark, rowsNum, dateTime) + Seq(true, false).foreach { after1582 => + benchmark.addCase(caseName(after1582), 1) { _ => + genDF(rowsNum, dateTime, after1582) + .write + .mode("overwrite") + .format("orc") + .save(getPath(path, dateTime, after1582)) + } + } + benchmark.run() + + val benchmark2 = new Benchmark( + s"Load ${dateTime}s from ORC", + rowsNum, + output = output) + Seq(true, false).foreach { after1582 => + Seq(false, true).foreach { vec => + benchmark2.addCase(caseName(after1582, vec = Some(vec)), 3) { _ => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vec.toString) { + spark + .read + .format("orc") + .load(getPath(path, dateTime, after1582)) + .noop() + } + } + } + } + benchmark2.run() + } + } + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala index f1315d48e4d99..285211e918d68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala @@ -84,8 +84,6 @@ object ExtractBenchmark extends SqlBasedBenchmark { } } - private case class Settings(fields: Seq[String], func: Seq[String], iterNum: Long) - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val N = 10000000L val datetimeFields = Seq( @@ -100,18 +98,16 @@ object ExtractBenchmark extends SqlBasedBenchmark { "HOUR", "MINUTE", "SECOND", "MILLISECONDS", "MICROSECONDS", "EPOCH") val settings = Map( - "timestamp" -> Settings(datetimeFields, Seq("extract", "date_part"), N), - "date" -> Settings(datetimeFields, Seq("extract", "date_part"), N), - "interval" -> Settings(intervalFields, Seq("date_part"), N)) + "timestamp" -> datetimeFields, + "date" -> datetimeFields, + "interval" -> intervalFields) - for { - (dataType, Settings(fields, funcs, iterNum)) <- settings - func <- funcs} { + for {(dataType, fields) <- settings; func <- Seq("extract", "date_part")} { val benchmark = new Benchmark(s"Invoke $func for $dataType", N, output = output) - run(benchmark, iterNum, s"cast to $dataType", castExpr(dataType)) - fields.foreach(run(benchmark, func, iterNum, _, dataType)) + run(benchmark, N, s"cast to $dataType", castExpr(dataType)) + fields.foreach(run(benchmark, func, N, _, dataType)) benchmark.run() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index bacd64efedc83..28e5082886b67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -40,7 +40,8 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class DDLParserSuite extends AnalysisTest with SharedSparkSession { - private lazy val parser = new SparkSqlParser(new SQLConf) + private lazy val parser = new SparkSqlParser(new SQLConf().copy( + SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED -> false)) private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { val e = intercept[ParseException] { 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 5a67dce0ffe6a..10ad8acc68937 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 @@ -2084,6 +2084,27 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } + test("SPARK-31163: acl/permission should handle non-existed path when truncating table") { + withSQLConf(SQLConf.TRUNCATE_TABLE_IGNORE_PERMISSION_ACL.key -> "false") { + withTable("tab1") { + sql("CREATE TABLE tab1 (col1 STRING, col2 INT) USING parquet PARTITIONED BY (col2)") + sql("INSERT INTO tab1 SELECT 'one', 1") + checkAnswer(spark.table("tab1"), Row("one", 1)) + val part = spark.sessionState.catalog.listPartitions(TableIdentifier("tab1")).head + val path = new File(part.location.getPath) + sql("TRUNCATE TABLE tab1") + // simulate incomplete/unsuccessful truncate + assert(path.exists()) + path.delete() + assert(!path.exists()) + // execute without java.io.FileNotFoundException + sql("TRUNCATE TABLE tab1") + // partition path should be re-created + assert(path.exists()) + } + } + } + test("create temporary view with mismatched schema") { withTable("tab1") { spark.range(10).write.saveAsTable("tab1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 9df3c336b2eee..2d6a5da6d67f7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -37,11 +37,13 @@ import org.apache.spark.sql.connector.catalog.TableChange.{UpdateColumnComment, import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.SimpleScanSource import org.apache.spark.sql.types.{CharType, DoubleType, HIVE_TYPE_STRING, IntegerType, LongType, MetadataBuilder, StringType, StructField, StructType} class PlanResolutionSuite extends AnalysisTest { import CatalystSqlParser._ + private val v1Format = classOf[SimpleScanSource].getName private val v2Format = classOf[FakeV2Provider].getName private val table: Table = { @@ -61,6 +63,15 @@ class PlanResolutionSuite extends AnalysisTest { val t = mock(classOf[CatalogTable]) when(t.schema).thenReturn(new StructType().add("i", "int").add("s", "string")) when(t.tableType).thenReturn(CatalogTableType.MANAGED) + when(t.provider).thenReturn(Some(v1Format)) + V1Table(t) + } + + private val v1HiveTable: V1Table = { + val t = mock(classOf[CatalogTable]) + when(t.schema).thenReturn(new StructType().add("i", "int").add("s", "string")) + when(t.tableType).thenReturn(CatalogTableType.MANAGED) + when(t.provider).thenReturn(Some("hive")) V1Table(t) } @@ -83,6 +94,7 @@ class PlanResolutionSuite extends AnalysisTest { invocation.getArgument[Identifier](0).name match { case "v1Table" => v1Table case "v1Table1" => v1Table + case "v1HiveTable" => v1HiveTable case "v2Table" => table case "v2Table1" => table case "v2TableWithAcceptAnySchemaCapability" => tableWithAcceptAnySchemaCapability @@ -540,7 +552,7 @@ class PlanResolutionSuite extends AnalysisTest { assert(ctas.catalog.name == "testcat") assert(ctas.tableName == Identifier.of(Array("mydb"), "table_name")) assert(ctas.properties == expectedProperties) - assert(ctas.writeOptions == Map("other" -> "20")) + assert(ctas.writeOptions.isEmpty) assert(ctas.partitioning.isEmpty) assert(ctas.ignoreIfExists) @@ -574,7 +586,7 @@ class PlanResolutionSuite extends AnalysisTest { assert(ctas.catalog.name == "testcat") assert(ctas.tableName == Identifier.of(Array("mydb"), "table_name")) assert(ctas.properties == expectedProperties) - assert(ctas.writeOptions == Map("other" -> "20")) + assert(ctas.writeOptions.isEmpty) assert(ctas.partitioning.isEmpty) assert(ctas.ignoreIfExists) @@ -1046,15 +1058,6 @@ class PlanResolutionSuite extends AnalysisTest { } assert(e2.getMessage.contains( "ALTER COLUMN with qualified column is only supported with v2 tables")) - - val sql5 = s"ALTER TABLE $tblName ALTER COLUMN i TYPE char(1)" - val builder = new MetadataBuilder - builder.putString(HIVE_TYPE_STRING, CharType(1).catalogString) - val newColumnWithCleanedType = StructField("i", StringType, true, builder.build()) - val expected5 = AlterTableChangeColumnCommand( - tableIdent, "i", newColumnWithCleanedType) - val parsed5 = parseAndResolve(sql5) - comparePlans(parsed5, expected5) } else { parsed1 match { case AlterTable(_, _, _: DataSourceV2Relation, changes) => @@ -1071,6 +1074,15 @@ class PlanResolutionSuite extends AnalysisTest { } } } + + val sql = s"ALTER TABLE v1HiveTable ALTER COLUMN i TYPE char(1)" + val builder = new MetadataBuilder + builder.putString(HIVE_TYPE_STRING, CharType(1).catalogString) + val newColumnWithCleanedType = StructField("i", StringType, true, builder.build()) + val expected = AlterTableChangeColumnCommand( + TableIdentifier("v1HiveTable", Some("default")), "i", newColumnWithCleanedType) + val parsed = parseAndResolve(sql) + comparePlans(parsed, expected) } test("alter table: alter column action is not specified") { @@ -1506,6 +1518,45 @@ class PlanResolutionSuite extends AnalysisTest { case l => fail("Expected unresolved MergeIntoTable, but got:\n" + l.treeString) } } + + test("SPARK-31147: forbid CHAR type in non-Hive tables") { + def checkFailure(t: String, provider: String): Unit = { + val types = Seq( + "CHAR(2)", + "ARRAY", + "MAP", + "MAP", + "STRUCT") + types.foreach { tpe => + intercept[AnalysisException] { + parseAndResolve(s"CREATE TABLE $t(col $tpe) USING $provider") + } + intercept[AnalysisException] { + parseAndResolve(s"REPLACE TABLE $t(col $tpe) USING $provider") + } + intercept[AnalysisException] { + parseAndResolve(s"CREATE OR REPLACE TABLE $t(col $tpe) USING $provider") + } + intercept[AnalysisException] { + parseAndResolve(s"ALTER TABLE $t ADD COLUMN col $tpe") + } + intercept[AnalysisException] { + parseAndResolve(s"ALTER TABLE $t ADD COLUMN col $tpe") + } + intercept[AnalysisException] { + parseAndResolve(s"ALTER TABLE $t ALTER COLUMN col TYPE $tpe") + } + intercept[AnalysisException] { + parseAndResolve(s"ALTER TABLE $t REPLACE COLUMNS (col $tpe)") + } + } + } + + checkFailure("v1Table", v1Format) + checkFailure("v2Table", v2Format) + checkFailure("testcat.tab", "foo") + } + // TODO: add tests for more commands. } @@ -1515,4 +1566,3 @@ object AsDataSourceV2Relation { case _ => None } } - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala index 7bd3213b378ce..a775a97895cfc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala @@ -26,15 +26,61 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructT class DataSourceStrategySuite extends PlanTest with SharedSparkSession { val attrInts = Seq( - 'cint.int + 'cint.int, + Symbol("c.int").int, + GetStructField('a.struct(StructType( + StructField("cstr", StringType, nullable = true) :: + StructField("cint", IntegerType, nullable = true) :: Nil)), 1, None), + GetStructField('a.struct(StructType( + StructField("c.int", IntegerType, nullable = true) :: + StructField("cstr", StringType, nullable = true) :: Nil)), 0, None), + GetStructField(Symbol("a.b").struct(StructType( + StructField("cstr1", StringType, nullable = true) :: + StructField("cstr2", StringType, nullable = true) :: + StructField("cint", IntegerType, nullable = true) :: Nil)), 2, None), + GetStructField(Symbol("a.b").struct(StructType( + StructField("c.int", IntegerType, nullable = true) :: Nil)), 0, None), + GetStructField(GetStructField('a.struct(StructType( + StructField("cstr1", StringType, nullable = true) :: + StructField("b", StructType(StructField("cint", IntegerType, nullable = true) :: + StructField("cstr2", StringType, nullable = true) :: Nil)) :: Nil)), 1, None), 0, None) ).zip(Seq( - "cint" + "cint", + "`c.int`", // single level field that contains `dot` in name + "a.cint", // two level nested field + "a.`c.int`", // two level nested field, and nested level contains `dot` + "`a.b`.cint", // two level nested field, and top level contains `dot` + "`a.b`.`c.int`", // two level nested field, and both levels contain `dot` + "a.b.cint" // three level nested field )) val attrStrs = Seq( - 'cstr.string + 'cstr.string, + Symbol("c.str").string, + GetStructField('a.struct(StructType( + StructField("cint", IntegerType, nullable = true) :: + StructField("cstr", StringType, nullable = true) :: Nil)), 1, None), + GetStructField('a.struct(StructType( + StructField("c.str", StringType, nullable = true) :: + StructField("cint", IntegerType, nullable = true) :: Nil)), 0, None), + GetStructField(Symbol("a.b").struct(StructType( + StructField("cint1", IntegerType, nullable = true) :: + StructField("cint2", IntegerType, nullable = true) :: + StructField("cstr", StringType, nullable = true) :: Nil)), 2, None), + GetStructField(Symbol("a.b").struct(StructType( + StructField("c.str", StringType, nullable = true) :: Nil)), 0, None), + GetStructField(GetStructField('a.struct(StructType( + StructField("cint1", IntegerType, nullable = true) :: + StructField("b", StructType(StructField("cstr", StringType, nullable = true) :: + StructField("cint2", IntegerType, nullable = true) :: Nil)) :: Nil)), 1, None), 0, None) ).zip(Seq( - "cstr" + "cstr", + "`c.str`", // single level field that contains `dot` in name + "a.cstr", // two level nested field + "a.`c.str`", // two level nested field, and nested level contains `dot` + "`a.b`.cstr", // two level nested field, and top level contains `dot` + "`a.b`.`c.str`", // two level nested field, and both levels contain `dot` + "a.b.cstr" // three level nested field )) test("translate simple expression") { attrInts.zip(attrStrs) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala index 553773e2555cf..ea15f1891b006 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -22,7 +22,11 @@ import java.net.URI import scala.collection.mutable -import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path, RawLocalFileSystem} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path, RawLocalFileSystem, RemoteIterator} +import org.apache.hadoop.fs.viewfs.ViewFileSystem +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{mock, when} import org.apache.spark.SparkException import org.apache.spark.metrics.source.HiveCatalogMetrics @@ -465,6 +469,25 @@ class FileIndexSuite extends SharedSparkSession { } } } + + test("SPARK-31047 - Improve file listing for ViewFileSystem") { + val path = mock(classOf[Path]) + val dfs = mock(classOf[ViewFileSystem]) + when(path.getFileSystem(any[Configuration])).thenReturn(dfs) + val statuses = + Seq( + new LocatedFileStatus( + new FileStatus(0, false, 0, 100, 0, + new Path("file")), Array(new BlockLocation())) + ) + when(dfs.listLocatedStatus(path)).thenReturn(new RemoteIterator[LocatedFileStatus] { + val iter = statuses.toIterator + override def hasNext: Boolean = iter.hasNext + override def next(): LocatedFileStatus = iter.next + }) + val fileIndex = new TestInMemoryFileIndex(spark, path) + assert(fileIndex.leafFileStatuses.toSeq == statuses) + } } object DeletionRaceFileSystem { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index a3d4905e82cee..80061dc84efbc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -301,6 +301,43 @@ abstract class SchemaPruningSuite checkAnswer(query, Row("Y.", 1) :: Row("X.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) } + testSchemaPruning("select explode of nested field of array of struct") { + // Config combinations + val configs = Seq((true, true), (true, false), (false, true), (false, false)) + + configs.foreach { case (nestedPruning, nestedPruningOnExpr) => + withSQLConf( + SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> nestedPruning.toString, + SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> nestedPruningOnExpr.toString) { + val query1 = spark.table("contacts") + .select(explode(col("friends.first"))) + if (nestedPruning) { + // If `NESTED_SCHEMA_PRUNING_ENABLED` is enabled, + // even disabling `NESTED_PRUNING_ON_EXPRESSIONS`, + // nested schema is still pruned at scan node. + checkScan(query1, "struct>>") + } else { + checkScan(query1, "struct>>") + } + checkAnswer(query1, Row("Susan") :: Nil) + + val query2 = spark.table("contacts") + .select(explode(col("friends.first")), col("friends.middle")) + if (nestedPruning) { + checkScan(query2, "struct>>") + } else { + checkScan(query2, "struct>>") + } + checkAnswer(query2, Row("Susan", Array("Z.")) :: Nil) + + val query3 = spark.table("contacts") + .select(explode(col("friends.first")), col("friends.middle"), col("friends.last")) + checkScan(query3, "struct>>") + checkAnswer(query3, Row("Susan", Array("Z."), Array("Smith")) :: Nil) + } + } + } + protected def testSchemaPruning(testName: String)(testThunk: => Unit): Unit = { test(s"Spark vectorized reader - without partition data column - $testName") { withSQLConf(vectorizedReaderEnabledKey -> "true") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala index e2abb39c986a7..53d287b32f8db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala @@ -238,7 +238,9 @@ object CSVBenchmark extends SqlBasedBenchmark { def timestampStr: Dataset[String] = { spark.range(0, rowsNum, 1, 1).mapPartitions { iter => - iter.map(i => s"1970-01-01T01:02:03.${100 + i % 100}Z") + iter.map { + i => s"1970-01-01T01:02:03.${i % 200}Z".stripSuffix(".0Z") + } }.select($"value".as("timestamp")).as[String] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 30ae9dc67a1e1..366cf11871fa0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -1080,7 +1080,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa .format("csv") .option("header", "true") .option("timestampFormat", "yyyy/MM/dd HH:mm") - .option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") .save(timestampsWithFormatPath) // This will load back the timestamps as string. @@ -1102,7 +1102,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa .option("header", "true") .option("inferSchema", "true") .option("timestampFormat", "yyyy/MM/dd HH:mm") - .option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") .load(timestampsWithFormatPath) checkAnswer(readBack, timestampsWithFormat) @@ -1897,6 +1897,20 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa assert(spark.read.csv(input).collect().toSet == Set(Row())) } + test("SPARK-31261: bad csv input with `columnNameCorruptRecord` should not cause NPE") { + val schema = StructType( + StructField("a", IntegerType) :: StructField("_corrupt_record", StringType) :: Nil) + val input = spark.createDataset(Seq("\u0000\u0000\u0001234")) + + checkAnswer( + spark.read + .option("columnNameOfCorruptRecord", "_corrupt_record") + .schema(schema) + .csv(input), + Row(null, null)) + assert(spark.read.csv(input).collect().toSet == Set(Row())) + } + test("field names of inferred schema shouldn't compare to the first row") { val input = Seq("1,2").toDS() val df = spark.read.option("enforceSchema", false).csv(input) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala new file mode 100644 index 0000000000000..d18a3088c4f2f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc.connection + +import java.sql.{Driver, DriverManager} +import javax.security.auth.login.Configuration + +import scala.collection.JavaConverters._ + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions} + +abstract class ConnectionProviderSuiteBase extends SparkFunSuite with BeforeAndAfterEach { + protected def registerDriver(driverClass: String): Driver = { + DriverRegistry.register(driverClass) + DriverManager.getDrivers.asScala.collectFirst { + case d if d.getClass.getCanonicalName == driverClass => d + }.get + } + + protected def options(url: String) = new JDBCOptions(Map[String, String]( + JDBCOptions.JDBC_URL -> url, + JDBCOptions.JDBC_TABLE_NAME -> "table", + JDBCOptions.JDBC_KEYTAB -> "/path/to/keytab", + JDBCOptions.JDBC_PRINCIPAL -> "principal" + )) + + override def afterEach(): Unit = { + try { + Configuration.setConfiguration(null) + } finally { + super.afterEach() + } + } + + protected def testSecureConnectionProvider(provider: SecureConnectionProvider): Unit = { + // Make sure no authentication for the database is set + assert(Configuration.getConfiguration.getAppConfigurationEntry(provider.appEntry) == null) + + // Make sure the first call sets authentication properly + val savedConfig = Configuration.getConfiguration + provider.setAuthenticationConfigIfNeeded() + val config = Configuration.getConfiguration + assert(savedConfig != config) + val appEntry = config.getAppConfigurationEntry(provider.appEntry) + assert(appEntry != null) + + // Make sure a second call is not modifying the existing authentication + provider.setAuthenticationConfigIfNeeded() + assert(config.getAppConfigurationEntry(provider.appEntry) === appEntry) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala new file mode 100644 index 0000000000000..70cad2097eb43 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala @@ -0,0 +1,27 @@ +/* + * 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.jdbc.connection + +class MariaDBConnectionProviderSuite extends ConnectionProviderSuiteBase { + test("setAuthenticationConfigIfNeeded must set authentication if not set") { + val driver = registerDriver(MariaDBConnectionProvider.driverClass) + val provider = new MariaDBConnectionProvider(driver, options("jdbc:mysql://localhost/mysql")) + + testSecureConnectionProvider(provider) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala index 59ff1c79bd064..8cef7652f9c54 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala @@ -17,69 +17,16 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection -import java.sql.{Driver, DriverManager} -import javax.security.auth.login.Configuration - -import scala.collection.JavaConverters._ - -import org.scalatest.BeforeAndAfterEach - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions} - -class PostgresConnectionProviderSuite extends SparkFunSuite with BeforeAndAfterEach { - private def options(url: String) = new JDBCOptions(Map[String, String]( - JDBCOptions.JDBC_URL -> url, - JDBCOptions.JDBC_TABLE_NAME -> "table", - JDBCOptions.JDBC_KEYTAB -> "/path/to/keytab", - JDBCOptions.JDBC_PRINCIPAL -> "principal" - )) - - override def afterEach(): Unit = { - try { - Configuration.setConfiguration(null) - } finally { - super.afterEach() - } - } - +class PostgresConnectionProviderSuite extends ConnectionProviderSuiteBase { test("setAuthenticationConfigIfNeeded must set authentication if not set") { - DriverRegistry.register(PostgresConnectionProvider.driverClass) - val driver = DriverManager.getDrivers.asScala.collectFirst { - case d if d.getClass.getCanonicalName == PostgresConnectionProvider.driverClass => d - }.get + val driver = registerDriver(PostgresConnectionProvider.driverClass) val defaultProvider = new PostgresConnectionProvider( driver, options("jdbc:postgresql://localhost/postgres")) val customProvider = new PostgresConnectionProvider( driver, options(s"jdbc:postgresql://localhost/postgres?jaasApplicationName=custompgjdbc")) assert(defaultProvider.appEntry !== customProvider.appEntry) - - // Make sure no authentication for postgres is set - assert(Configuration.getConfiguration.getAppConfigurationEntry( - defaultProvider.appEntry) == null) - assert(Configuration.getConfiguration.getAppConfigurationEntry( - customProvider.appEntry) == null) - - // Make sure the first call sets authentication properly - val savedConfig = Configuration.getConfiguration - defaultProvider.setAuthenticationConfigIfNeeded() - val defaultConfig = Configuration.getConfiguration - assert(savedConfig != defaultConfig) - val defaultAppEntry = defaultConfig.getAppConfigurationEntry(defaultProvider.appEntry) - assert(defaultAppEntry != null) - customProvider.setAuthenticationConfigIfNeeded() - val customConfig = Configuration.getConfiguration - assert(savedConfig != customConfig) - assert(defaultConfig != customConfig) - val customAppEntry = customConfig.getAppConfigurationEntry(customProvider.appEntry) - assert(customAppEntry != null) - - // Make sure a second call is not modifying the existing authentication - defaultProvider.setAuthenticationConfigIfNeeded() - customProvider.setAuthenticationConfigIfNeeded() - assert(customConfig == Configuration.getConfiguration) - assert(defaultConfig.getAppConfigurationEntry(defaultProvider.appEntry) === defaultAppEntry) - assert(customConfig.getAppConfigurationEntry(customProvider.appEntry) === customAppEntry) + testSecureConnectionProvider(defaultProvider) + testSecureConnectionProvider(customProvider) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala index bcecaccc8cc89..56930880ed5da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala @@ -445,7 +445,9 @@ object JsonBenchmark extends SqlBasedBenchmark { def timestampStr: Dataset[String] = { spark.range(0, rowsNum, 1, 1).mapPartitions { iter => - iter.map(i => s"""{"timestamp":"1970-01-01T01:02:03.${100 + i % 100}Z"}""") + iter.map { i => + s"""{"timestamp":"1970-01-01T01:02:03.${i % 200}Z"}""".stripSuffix(".0Z") + } }.select($"value".as("timestamp")).as[String] } 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 9d026527a3795..85584a99a4131 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 @@ -21,7 +21,7 @@ import java.io._ import java.nio.charset.{Charset, StandardCharsets, UnsupportedCharsetException} import java.nio.file.Files import java.sql.{Date, Timestamp} -import java.time.{LocalDate, LocalDateTime, ZoneId} +import java.time.LocalDate import java.util.Locale import com.fasterxml.jackson.core.JsonFactory @@ -35,6 +35,7 @@ import org.apache.spark.sql.{functions => F, _} import org.apache.spark.sql.catalyst.json._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.ExternalRDD +import org.apache.spark.sql.execution.adaptive.AdaptiveTestUtils.assertExceptionMessage import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -1329,7 +1330,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson test("SPARK-6245 JsonInferSchema.infer on empty RDD") { // This is really a test that it doesn't throw an exception - val options = new JSONOptions(Map.empty[String, String], "GMT") + val options = new JSONOptions(Map.empty[String, String], "UTC") val emptySchema = new JsonInferSchema(options).infer( empty.rdd, CreateJacksonParser.string) @@ -1356,7 +1357,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson } test("SPARK-8093 Erase empty structs") { - val options = new JSONOptions(Map.empty[String, String], "GMT") + val options = new JSONOptions(Map.empty[String, String], "UTC") val emptySchema = new JsonInferSchema(options).infer( emptyRecords.rdd, CreateJacksonParser.string) @@ -1730,7 +1731,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson timestampsWithFormat.write .format("json") .option("timestampFormat", "yyyy/MM/dd HH:mm") - .option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") .save(timestampsWithFormatPath) // This will load back the timestamps as string. @@ -1748,7 +1749,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson val readBack = spark.read .schema(customSchema) .option("timestampFormat", "yyyy/MM/dd HH:mm") - .option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") .json(timestampsWithFormatPath) checkAnswer(readBack, timestampsWithFormat) @@ -2192,9 +2193,8 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson .json(testFile(fileName)) .count() } - val errMsg = exception.getMessage - assert(errMsg.contains("Malformed records are detected in record parsing")) + assertExceptionMessage(exception, "Malformed records are detected in record parsing") } def checkEncoding(expectedEncoding: String, pathToJsonFiles: String, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 5f495263b30e3..60f278b8e5bb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -34,6 +34,7 @@ import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.adaptive.AdaptiveTestUtils.assertExceptionMessage import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, RecordReaderIterator} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -595,22 +596,22 @@ abstract class OrcQueryTest extends OrcTest { } withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") { - val m1 = intercept[SparkException] { + val e1 = intercept[SparkException] { testIgnoreCorruptFiles() - }.getMessage - assert(m1.contains("Malformed ORC file")) - val m2 = intercept[SparkException] { + } + assertExceptionMessage(e1, "Malformed ORC file") + val e2 = intercept[SparkException] { testIgnoreCorruptFilesWithoutSchemaInfer() - }.getMessage - assert(m2.contains("Malformed ORC file")) - val m3 = intercept[SparkException] { + } + assertExceptionMessage(e2, "Malformed ORC file") + val e3 = intercept[SparkException] { testAllCorruptFiles() - }.getMessage - assert(m3.contains("Could not read footer for file")) - val m4 = intercept[SparkException] { + } + assertExceptionMessage(e3, "Could not read footer for file") + val e4 = intercept[SparkException] { testAllCorruptFilesWithoutSchemaInfer() - }.getMessage - assert(m4.contains("Malformed ORC file")) + } + assertExceptionMessage(e4, "Malformed ORC file") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 1e27593584786..0b7500c012686 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.orc import java.io.File import java.nio.charset.StandardCharsets.UTF_8 -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.util.Locale import org.apache.hadoop.conf.Configuration @@ -482,6 +482,60 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } } + + test("SPARK-31238: compatibility with Spark 2.4 in reading dates") { + Seq(false, true).foreach { vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + readResourceOrcFile("test-data/before_1582_date_v2_4.snappy.orc"), + Row(java.sql.Date.valueOf("1200-01-01"))) + } + } + } + + test("SPARK-31238: rebasing dates in write") { + withTempPath { dir => + val path = dir.getAbsolutePath + Seq("1001-01-01").toDF("dateS") + .select($"dateS".cast("date").as("date")) + .write + .orc(path) + + Seq(false, true).foreach { vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer(spark.read.orc(path), Row(Date.valueOf("1001-01-01"))) + } + } + } + } + + test("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") { + Seq(false, true).foreach { vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + readResourceOrcFile("test-data/before_1582_ts_v2_4.snappy.orc"), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"))) + } + } + } + + test("SPARK-31284: rebasing timestamps in write") { + withTempPath { dir => + val path = dir.getAbsolutePath + Seq("1001-01-01 01:02:03.123456").toDF("tsS") + .select($"tsS".cast("timestamp").as("ts")) + .write + .orc(path) + + Seq(false, true).foreach { vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + spark.read.orc(path), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"))) + } + } + } + } } class OrcSourceSuite extends OrcSuite with SharedSparkSession { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index 388744bd0fd6e..e929f904c798d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -22,6 +22,7 @@ import java.io.File import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag +import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ @@ -133,4 +134,13 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor throw new AnalysisException("Can not match OrcTable in the query.") } } + + protected def readResourceOrcFile(name: String): DataFrame = { + val url = Thread.currentThread().getContextClassLoader.getResource(name) + // Copy to avoid URISyntaxException when `sql/hive` accesses the resources in `sql/core` + val file = File.createTempFile("orc-test", ".orc") + file.deleteOnExit(); + FileUtils.copyURLToFile(url, file) + spark.read.orc(file.getAbsolutePath) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 4e0c1c2dbe601..d1161e33b0941 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -103,22 +103,42 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df) } - private def checkBinaryFilterPredicate - (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Seq[Row]) - (implicit df: DataFrame): Unit = { - def checkBinaryAnswer(df: DataFrame, expected: Seq[Row]) = { - assertResult(expected.map(_.getAs[Array[Byte]](0).mkString(",")).sorted) { - df.rdd.map(_.getAs[Array[Byte]](0).mkString(",")).collect().toSeq.sorted - } + /** + * Takes single level `inputDF` dataframe to generate multi-level nested + * dataframes as new test data. + */ + private def withNestedDataFrame(inputDF: DataFrame) + (runTest: (DataFrame, String, Any => Any) => Unit): Unit = { + assert(inputDF.schema.fields.length == 1) + assert(!inputDF.schema.fields.head.dataType.isInstanceOf[StructType]) + val df = inputDF.toDF("temp") + Seq( + ( + df.withColumnRenamed("temp", "a"), + "a", // zero nesting + (x: Any) => x), + ( + df.withColumn("a", struct(df("temp") as "b")).drop("temp"), + "a.b", // one level nesting + (x: Any) => Row(x)), + ( + df.withColumn("a", struct(struct(df("temp") as "c") as "b")).drop("temp"), + "a.b.c", // two level nesting + (x: Any) => Row(Row(x)) + ), + ( + df.withColumnRenamed("temp", "a.b"), + "`a.b`", // zero nesting with column name containing `dots` + (x: Any) => x + ), + ( + df.withColumn("a.b", struct(df("temp") as "c.d") ).drop("temp"), + "`a.b`.`c.d`", // one level nesting with column names containing `dots` + (x: Any) => Row(x) + ) + ).foreach { case (df, colName, resultFun) => + runTest(df, colName, resultFun) } - - checkFilterPredicate(df, predicate, filterClass, checkBinaryAnswer _, expected) - } - - private def checkBinaryFilterPredicate - (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Array[Byte]) - (implicit df: DataFrame): Unit = { - checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df) } private def testTimestampPushdown(data: Seq[Timestamp]): Unit = { @@ -128,36 +148,38 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared val ts3 = data(2) val ts4 = data(3) - withParquetDataFrame(data.map(i => Tuple1(i))) { implicit df => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], data.map(i => Row.apply(i))) - - checkFilterPredicate('_1 === ts1, classOf[Eq[_]], ts1) - checkFilterPredicate('_1 <=> ts1, classOf[Eq[_]], ts1) - checkFilterPredicate('_1 =!= ts1, classOf[NotEq[_]], - Seq(ts2, ts3, ts4).map(i => Row.apply(i))) - - checkFilterPredicate('_1 < ts2, classOf[Lt[_]], ts1) - checkFilterPredicate('_1 > ts1, classOf[Gt[_]], Seq(ts2, ts3, ts4).map(i => Row.apply(i))) - checkFilterPredicate('_1 <= ts1, classOf[LtEq[_]], ts1) - checkFilterPredicate('_1 >= ts4, classOf[GtEq[_]], ts4) - - checkFilterPredicate(Literal(ts1) === '_1, classOf[Eq[_]], ts1) - checkFilterPredicate(Literal(ts1) <=> '_1, classOf[Eq[_]], ts1) - checkFilterPredicate(Literal(ts2) > '_1, classOf[Lt[_]], ts1) - checkFilterPredicate(Literal(ts3) < '_1, classOf[Gt[_]], ts4) - checkFilterPredicate(Literal(ts1) >= '_1, classOf[LtEq[_]], ts1) - checkFilterPredicate(Literal(ts4) <= '_1, classOf[GtEq[_]], ts4) - - checkFilterPredicate(!('_1 < ts4), classOf[GtEq[_]], ts4) - checkFilterPredicate('_1 < ts2 || '_1 > ts3, classOf[Operators.Or], Seq(Row(ts1), Row(ts4))) - } - } - - private def testDecimalPushDown(data: DataFrame)(f: DataFrame => Unit): Unit = { - withTempPath { file => - data.write.parquet(file.getCanonicalPath) - readParquetFile(file.toString)(f) + import testImplicits._ + withNestedDataFrame(data.map(i => Tuple1(i)).toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val tsAttr = df(colName).expr + assert(df(colName).expr.dataType === TimestampType) + + checkFilterPredicate(tsAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(tsAttr.isNotNull, classOf[NotEq[_]], + data.map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(tsAttr === ts1, classOf[Eq[_]], resultFun(ts1)) + checkFilterPredicate(tsAttr <=> ts1, classOf[Eq[_]], resultFun(ts1)) + checkFilterPredicate(tsAttr =!= ts1, classOf[NotEq[_]], + Seq(ts2, ts3, ts4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(tsAttr < ts2, classOf[Lt[_]], resultFun(ts1)) + checkFilterPredicate(tsAttr > ts1, classOf[Gt[_]], + Seq(ts2, ts3, ts4).map(i => Row.apply(resultFun(i)))) + checkFilterPredicate(tsAttr <= ts1, classOf[LtEq[_]], resultFun(ts1)) + checkFilterPredicate(tsAttr >= ts4, classOf[GtEq[_]], resultFun(ts4)) + + checkFilterPredicate(Literal(ts1) === tsAttr, classOf[Eq[_]], resultFun(ts1)) + checkFilterPredicate(Literal(ts1) <=> tsAttr, classOf[Eq[_]], resultFun(ts1)) + checkFilterPredicate(Literal(ts2) > tsAttr, classOf[Lt[_]], resultFun(ts1)) + checkFilterPredicate(Literal(ts3) < tsAttr, classOf[Gt[_]], resultFun(ts4)) + checkFilterPredicate(Literal(ts1) >= tsAttr, classOf[LtEq[_]], resultFun(ts1)) + checkFilterPredicate(Literal(ts4) <= tsAttr, classOf[GtEq[_]], resultFun(ts4)) + + checkFilterPredicate(!(tsAttr < ts4), classOf[GtEq[_]], resultFun(ts4)) + checkFilterPredicate(tsAttr < ts2 || tsAttr > ts3, classOf[Operators.Or], + Seq(Row(resultFun(ts1)), Row(resultFun(ts4)))) + } } } @@ -187,201 +209,273 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } test("filter pushdown - boolean") { - withParquetDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], Seq(Row(true), Row(false))) - - checkFilterPredicate('_1 === true, classOf[Eq[_]], true) - checkFilterPredicate('_1 <=> true, classOf[Eq[_]], true) - checkFilterPredicate('_1 =!= true, classOf[NotEq[_]], false) + val data = (true :: false :: Nil).map(b => Tuple1.apply(Option(b))) + import testImplicits._ + withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val booleanAttr = df(colName).expr + assert(df(colName).expr.dataType === BooleanType) + + checkFilterPredicate(booleanAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(booleanAttr.isNotNull, classOf[NotEq[_]], + Seq(Row(resultFun(true)), Row(resultFun(false)))) + + checkFilterPredicate(booleanAttr === true, classOf[Eq[_]], resultFun(true)) + checkFilterPredicate(booleanAttr <=> true, classOf[Eq[_]], resultFun(true)) + checkFilterPredicate(booleanAttr =!= true, classOf[NotEq[_]], resultFun(false)) + } } } test("filter pushdown - tinyint") { - withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toByte)))) { implicit df => - assert(df.schema.head.dataType === ByteType) - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 === 1.toByte, classOf[Eq[_]], 1) - checkFilterPredicate('_1 <=> 1.toByte, classOf[Eq[_]], 1) - checkFilterPredicate('_1 =!= 1.toByte, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 < 2.toByte, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3.toByte, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1.toByte, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4.toByte, classOf[GtEq[_]], 4) - - checkFilterPredicate(Literal(1.toByte) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(1.toByte) <=> '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2.toByte) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3.toByte) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1.toByte) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4.toByte) <= '_1, classOf[GtEq[_]], 4) - - checkFilterPredicate(!('_1 < 4.toByte), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 < 2.toByte || '_1 > 3.toByte, - classOf[Operators.Or], Seq(Row(1), Row(4))) + val data = (1 to 4).map(i => Tuple1(Option(i.toByte))) + import testImplicits._ + withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val tinyIntAttr = df(colName).expr + assert(df(colName).expr.dataType === ByteType) + + checkFilterPredicate(tinyIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(tinyIntAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(tinyIntAttr === 1.toByte, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(tinyIntAttr <=> 1.toByte, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(tinyIntAttr =!= 1.toByte, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(tinyIntAttr < 2.toByte, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(tinyIntAttr > 3.toByte, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(tinyIntAttr <= 1.toByte, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(tinyIntAttr >= 4.toByte, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1.toByte) === tinyIntAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1.toByte) <=> tinyIntAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2.toByte) > tinyIntAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3.toByte) < tinyIntAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1.toByte) >= tinyIntAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4.toByte) <= tinyIntAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(tinyIntAttr < 4.toByte), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(tinyIntAttr < 2.toByte || tinyIntAttr > 3.toByte, + classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4)))) + } } } test("filter pushdown - smallint") { - withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toShort)))) { implicit df => - assert(df.schema.head.dataType === ShortType) - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 === 1.toShort, classOf[Eq[_]], 1) - checkFilterPredicate('_1 <=> 1.toShort, classOf[Eq[_]], 1) - checkFilterPredicate('_1 =!= 1.toShort, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 < 2.toShort, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3.toShort, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1.toShort, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4.toShort, classOf[GtEq[_]], 4) - - checkFilterPredicate(Literal(1.toShort) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(1.toShort) <=> '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2.toShort) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3.toShort) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1.toShort) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4.toShort) <= '_1, classOf[GtEq[_]], 4) - - checkFilterPredicate(!('_1 < 4.toShort), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 < 2.toShort || '_1 > 3.toShort, - classOf[Operators.Or], Seq(Row(1), Row(4))) + val data = (1 to 4).map(i => Tuple1(Option(i.toShort))) + import testImplicits._ + withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val smallIntAttr = df(colName).expr + assert(df(colName).expr.dataType === ShortType) + + checkFilterPredicate(smallIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(smallIntAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(smallIntAttr === 1.toShort, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(smallIntAttr <=> 1.toShort, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(smallIntAttr =!= 1.toShort, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(smallIntAttr < 2.toShort, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(smallIntAttr > 3.toShort, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(smallIntAttr <= 1.toShort, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(smallIntAttr >= 4.toShort, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1.toShort) === smallIntAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1.toShort) <=> smallIntAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2.toShort) > smallIntAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3.toShort) < smallIntAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1.toShort) >= smallIntAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4.toShort) <= smallIntAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(smallIntAttr < 4.toShort), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(smallIntAttr < 2.toShort || smallIntAttr > 3.toShort, + classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4)))) + } } } test("filter pushdown - integer") { - withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) - - checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) - - checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) + val data = (1 to 4).map(i => Tuple1(Option(i))) + import testImplicits._ + withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val intAttr = df(colName).expr + assert(df(colName).expr.dataType === IntegerType) + + checkFilterPredicate(intAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(intAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(intAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(intAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(intAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(intAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(intAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(intAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(intAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === intAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> intAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > intAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < intAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= intAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= intAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(intAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(intAttr < 2 || intAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) + } } } test("filter pushdown - long") { - withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) - - checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) - - checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) + val data = (1 to 4).map(i => Tuple1(Option(i.toLong))) + import testImplicits._ + withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val longAttr = df(colName).expr + assert(df(colName).expr.dataType === LongType) + + checkFilterPredicate(longAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(longAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(longAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(longAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(longAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(longAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(longAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(longAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(longAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === longAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> longAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > longAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < longAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= longAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= longAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(longAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(longAttr < 2 || longAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) + } } } test("filter pushdown - float") { - withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) - - checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) - - checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) + val data = (1 to 4).map(i => Tuple1(Option(i.toFloat))) + import testImplicits._ + withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val floatAttr = df(colName).expr + assert(df(colName).expr.dataType === FloatType) + + checkFilterPredicate(floatAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(floatAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(floatAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(floatAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(floatAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(floatAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(floatAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(floatAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(floatAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === floatAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> floatAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > floatAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < floatAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= floatAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= floatAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(floatAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(floatAttr < 2 || floatAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) + } } } test("filter pushdown - double") { - withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - - checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) - - checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) - - checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) + val data = (1 to 4).map(i => Tuple1(Option(i.toDouble))) + import testImplicits._ + withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val doubleAttr = df(colName).expr + assert(df(colName).expr.dataType === DoubleType) + + checkFilterPredicate(doubleAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(doubleAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(doubleAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(doubleAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(doubleAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(doubleAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(doubleAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(doubleAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(doubleAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === doubleAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> doubleAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > doubleAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < doubleAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= doubleAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= doubleAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(doubleAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(doubleAttr < 2 || doubleAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) + } } } test("filter pushdown - string") { - withParquetDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate( - '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.toString))) - - checkFilterPredicate('_1 === "1", classOf[Eq[_]], "1") - checkFilterPredicate('_1 <=> "1", classOf[Eq[_]], "1") - checkFilterPredicate( - '_1 =!= "1", classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.toString))) - - checkFilterPredicate('_1 < "2", classOf[Lt[_]], "1") - checkFilterPredicate('_1 > "3", classOf[Gt[_]], "4") - checkFilterPredicate('_1 <= "1", classOf[LtEq[_]], "1") - checkFilterPredicate('_1 >= "4", classOf[GtEq[_]], "4") - - checkFilterPredicate(Literal("1") === '_1, classOf[Eq[_]], "1") - checkFilterPredicate(Literal("1") <=> '_1, classOf[Eq[_]], "1") - checkFilterPredicate(Literal("2") > '_1, classOf[Lt[_]], "1") - checkFilterPredicate(Literal("3") < '_1, classOf[Gt[_]], "4") - checkFilterPredicate(Literal("1") >= '_1, classOf[LtEq[_]], "1") - checkFilterPredicate(Literal("4") <= '_1, classOf[GtEq[_]], "4") - - checkFilterPredicate(!('_1 < "4"), classOf[GtEq[_]], "4") - checkFilterPredicate('_1 < "2" || '_1 > "3", classOf[Operators.Or], Seq(Row("1"), Row("4"))) + val data = (1 to 4).map(i => Tuple1(Option(i.toString))) + import testImplicits._ + withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val stringAttr = df(colName).expr + assert(df(colName).expr.dataType === StringType) + + checkFilterPredicate(stringAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(stringAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i.toString)))) + + checkFilterPredicate(stringAttr === "1", classOf[Eq[_]], resultFun("1")) + checkFilterPredicate(stringAttr <=> "1", classOf[Eq[_]], resultFun("1")) + checkFilterPredicate(stringAttr =!= "1", classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i.toString)))) + + checkFilterPredicate(stringAttr < "2", classOf[Lt[_]], resultFun("1")) + checkFilterPredicate(stringAttr > "3", classOf[Gt[_]], resultFun("4")) + checkFilterPredicate(stringAttr <= "1", classOf[LtEq[_]], resultFun("1")) + checkFilterPredicate(stringAttr >= "4", classOf[GtEq[_]], resultFun("4")) + + checkFilterPredicate(Literal("1") === stringAttr, classOf[Eq[_]], resultFun("1")) + checkFilterPredicate(Literal("1") <=> stringAttr, classOf[Eq[_]], resultFun("1")) + checkFilterPredicate(Literal("2") > stringAttr, classOf[Lt[_]], resultFun("1")) + checkFilterPredicate(Literal("3") < stringAttr, classOf[Gt[_]], resultFun("4")) + checkFilterPredicate(Literal("1") >= stringAttr, classOf[LtEq[_]], resultFun("1")) + checkFilterPredicate(Literal("4") <= stringAttr, classOf[GtEq[_]], resultFun("4")) + + checkFilterPredicate(!(stringAttr < "4"), classOf[GtEq[_]], resultFun("4")) + checkFilterPredicate(stringAttr < "2" || stringAttr > "3", classOf[Operators.Or], + Seq(Row(resultFun("1")), Row(resultFun("4")))) + } } } @@ -390,32 +484,39 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) } - withParquetDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkBinaryFilterPredicate('_1 === 1.b, classOf[Eq[_]], 1.b) - checkBinaryFilterPredicate('_1 <=> 1.b, classOf[Eq[_]], 1.b) - - checkBinaryFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkBinaryFilterPredicate( - '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.b)).toSeq) - - checkBinaryFilterPredicate( - '_1 =!= 1.b, classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.b)).toSeq) - - checkBinaryFilterPredicate('_1 < 2.b, classOf[Lt[_]], 1.b) - checkBinaryFilterPredicate('_1 > 3.b, classOf[Gt[_]], 4.b) - checkBinaryFilterPredicate('_1 <= 1.b, classOf[LtEq[_]], 1.b) - checkBinaryFilterPredicate('_1 >= 4.b, classOf[GtEq[_]], 4.b) - - checkBinaryFilterPredicate(Literal(1.b) === '_1, classOf[Eq[_]], 1.b) - checkBinaryFilterPredicate(Literal(1.b) <=> '_1, classOf[Eq[_]], 1.b) - checkBinaryFilterPredicate(Literal(2.b) > '_1, classOf[Lt[_]], 1.b) - checkBinaryFilterPredicate(Literal(3.b) < '_1, classOf[Gt[_]], 4.b) - checkBinaryFilterPredicate(Literal(1.b) >= '_1, classOf[LtEq[_]], 1.b) - checkBinaryFilterPredicate(Literal(4.b) <= '_1, classOf[GtEq[_]], 4.b) - - checkBinaryFilterPredicate(!('_1 < 4.b), classOf[GtEq[_]], 4.b) - checkBinaryFilterPredicate( - '_1 < 2.b || '_1 > 3.b, classOf[Operators.Or], Seq(Row(1.b), Row(4.b))) + val data = (1 to 4).map(i => Tuple1(Option(i.b))) + import testImplicits._ + withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val binaryAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === BinaryType) + + checkFilterPredicate(binaryAttr === 1.b, classOf[Eq[_]], resultFun(1.b)) + checkFilterPredicate(binaryAttr <=> 1.b, classOf[Eq[_]], resultFun(1.b)) + + checkFilterPredicate(binaryAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(binaryAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i.b)))) + + checkFilterPredicate(binaryAttr =!= 1.b, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i.b)))) + + checkFilterPredicate(binaryAttr < 2.b, classOf[Lt[_]], resultFun(1.b)) + checkFilterPredicate(binaryAttr > 3.b, classOf[Gt[_]], resultFun(4.b)) + checkFilterPredicate(binaryAttr <= 1.b, classOf[LtEq[_]], resultFun(1.b)) + checkFilterPredicate(binaryAttr >= 4.b, classOf[GtEq[_]], resultFun(4.b)) + + checkFilterPredicate(Literal(1.b) === binaryAttr, classOf[Eq[_]], resultFun(1.b)) + checkFilterPredicate(Literal(1.b) <=> binaryAttr, classOf[Eq[_]], resultFun(1.b)) + checkFilterPredicate(Literal(2.b) > binaryAttr, classOf[Lt[_]], resultFun(1.b)) + checkFilterPredicate(Literal(3.b) < binaryAttr, classOf[Gt[_]], resultFun(4.b)) + checkFilterPredicate(Literal(1.b) >= binaryAttr, classOf[LtEq[_]], resultFun(1.b)) + checkFilterPredicate(Literal(4.b) <= binaryAttr, classOf[GtEq[_]], resultFun(4.b)) + + checkFilterPredicate(!(binaryAttr < 4.b), classOf[GtEq[_]], resultFun(4.b)) + checkFilterPredicate(binaryAttr < 2.b || binaryAttr > 3.b, classOf[Operators.Or], + Seq(Row(resultFun(1.b)), Row(resultFun(4.b)))) + } } } @@ -424,40 +525,53 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared def date: Date = Date.valueOf(s) } - val data = Seq("2018-03-18", "2018-03-19", "2018-03-20", "2018-03-21") - - withParquetDataFrame(data.map(i => Tuple1(i.date))) { implicit df => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], data.map(i => Row.apply(i.date))) - - checkFilterPredicate('_1 === "2018-03-18".date, classOf[Eq[_]], "2018-03-18".date) - checkFilterPredicate('_1 <=> "2018-03-18".date, classOf[Eq[_]], "2018-03-18".date) - checkFilterPredicate('_1 =!= "2018-03-18".date, classOf[NotEq[_]], - Seq("2018-03-19", "2018-03-20", "2018-03-21").map(i => Row.apply(i.date))) - - checkFilterPredicate('_1 < "2018-03-19".date, classOf[Lt[_]], "2018-03-18".date) - checkFilterPredicate('_1 > "2018-03-20".date, classOf[Gt[_]], "2018-03-21".date) - checkFilterPredicate('_1 <= "2018-03-18".date, classOf[LtEq[_]], "2018-03-18".date) - checkFilterPredicate('_1 >= "2018-03-21".date, classOf[GtEq[_]], "2018-03-21".date) - - checkFilterPredicate( - Literal("2018-03-18".date) === '_1, classOf[Eq[_]], "2018-03-18".date) - checkFilterPredicate( - Literal("2018-03-18".date) <=> '_1, classOf[Eq[_]], "2018-03-18".date) - checkFilterPredicate( - Literal("2018-03-19".date) > '_1, classOf[Lt[_]], "2018-03-18".date) - checkFilterPredicate( - Literal("2018-03-20".date) < '_1, classOf[Gt[_]], "2018-03-21".date) - checkFilterPredicate( - Literal("2018-03-18".date) >= '_1, classOf[LtEq[_]], "2018-03-18".date) - checkFilterPredicate( - Literal("2018-03-21".date) <= '_1, classOf[GtEq[_]], "2018-03-21".date) - - checkFilterPredicate(!('_1 < "2018-03-21".date), classOf[GtEq[_]], "2018-03-21".date) - checkFilterPredicate( - '_1 < "2018-03-19".date || '_1 > "2018-03-20".date, - classOf[Operators.Or], - Seq(Row("2018-03-18".date), Row("2018-03-21".date))) + val data = Seq("2018-03-18", "2018-03-19", "2018-03-20", "2018-03-21").map(_.date) + import testImplicits._ + withNestedDataFrame(data.map(i => Tuple1(i)).toDF()) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val dateAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === DateType) + + checkFilterPredicate(dateAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(dateAttr.isNotNull, classOf[NotEq[_]], + data.map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(dateAttr === "2018-03-18".date, classOf[Eq[_]], + resultFun("2018-03-18".date)) + checkFilterPredicate(dateAttr <=> "2018-03-18".date, classOf[Eq[_]], + resultFun("2018-03-18".date)) + checkFilterPredicate(dateAttr =!= "2018-03-18".date, classOf[NotEq[_]], + Seq("2018-03-19", "2018-03-20", "2018-03-21").map(i => Row.apply(resultFun(i.date)))) + + checkFilterPredicate(dateAttr < "2018-03-19".date, classOf[Lt[_]], + resultFun("2018-03-18".date)) + checkFilterPredicate(dateAttr > "2018-03-20".date, classOf[Gt[_]], + resultFun("2018-03-21".date)) + checkFilterPredicate(dateAttr <= "2018-03-18".date, classOf[LtEq[_]], + resultFun("2018-03-18".date)) + checkFilterPredicate(dateAttr >= "2018-03-21".date, classOf[GtEq[_]], + resultFun("2018-03-21".date)) + + checkFilterPredicate(Literal("2018-03-18".date) === dateAttr, classOf[Eq[_]], + resultFun("2018-03-18".date)) + checkFilterPredicate(Literal("2018-03-18".date) <=> dateAttr, classOf[Eq[_]], + resultFun("2018-03-18".date)) + checkFilterPredicate(Literal("2018-03-19".date) > dateAttr, classOf[Lt[_]], + resultFun("2018-03-18".date)) + checkFilterPredicate(Literal("2018-03-20".date) < dateAttr, classOf[Gt[_]], + resultFun("2018-03-21".date)) + checkFilterPredicate(Literal("2018-03-18".date) >= dateAttr, classOf[LtEq[_]], + resultFun("2018-03-18".date)) + checkFilterPredicate(Literal("2018-03-21".date) <= dateAttr, classOf[GtEq[_]], + resultFun("2018-03-21".date)) + + checkFilterPredicate(!(dateAttr < "2018-03-21".date), classOf[GtEq[_]], + resultFun("2018-03-21".date)) + checkFilterPredicate( + dateAttr < "2018-03-19".date || dateAttr > "2018-03-20".date, + classOf[Operators.Or], + Seq(Row(resultFun("2018-03-18".date)), Row(resultFun("2018-03-21".date)))) + } } } @@ -485,7 +599,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared // spark.sql.parquet.outputTimestampType = INT96 doesn't support pushdown withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> ParquetOutputTimestampType.INT96.toString) { - withParquetDataFrame(millisData.map(i => Tuple1(i))) { implicit df => + import testImplicits._ + withParquetDataFrame(millisData.map(i => Tuple1(i)).toDF()) { implicit df => val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) assertResult(None) { createParquetFilters(schema).createFilter(sources.IsNull("_1")) @@ -502,33 +617,39 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared (false, DecimalType.MAX_PRECISION) // binaryWriterUsingUnscaledBytes ).foreach { case (legacyFormat, precision) => withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> legacyFormat.toString) { - val schema = StructType.fromDDL(s"a decimal($precision, 2)") val rdd = spark.sparkContext.parallelize((1 to 4).map(i => Row(new java.math.BigDecimal(i)))) - val dataFrame = spark.createDataFrame(rdd, schema) - testDecimalPushDown(dataFrame) { implicit df => - assert(df.schema === schema) - checkFilterPredicate('a.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('a.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - - checkFilterPredicate('a === 1, classOf[Eq[_]], 1) - checkFilterPredicate('a <=> 1, classOf[Eq[_]], 1) - checkFilterPredicate('a =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - - checkFilterPredicate('a < 2, classOf[Lt[_]], 1) - checkFilterPredicate('a > 3, classOf[Gt[_]], 4) - checkFilterPredicate('a <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate('a >= 4, classOf[GtEq[_]], 4) - - checkFilterPredicate(Literal(1) === 'a, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(1) <=> 'a, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > 'a, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < 'a, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= 'a, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= 'a, classOf[GtEq[_]], 4) - - checkFilterPredicate(!('a < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('a < 2 || 'a > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) + val dataFrame = spark.createDataFrame(rdd, StructType.fromDDL(s"a decimal($precision, 2)")) + withNestedDataFrame(dataFrame) { case (inputDF, colName, resultFun) => + withParquetDataFrame(inputDF) { implicit df => + val decimalAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === DecimalType(precision, 2)) + + checkFilterPredicate(decimalAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate(decimalAttr.isNotNull, classOf[NotEq[_]], + (1 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(decimalAttr === 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(decimalAttr <=> 1, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(decimalAttr =!= 1, classOf[NotEq[_]], + (2 to 4).map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate(decimalAttr < 2, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(decimalAttr > 3, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(decimalAttr <= 1, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(decimalAttr >= 4, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(Literal(1) === decimalAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(1) <=> decimalAttr, classOf[Eq[_]], resultFun(1)) + checkFilterPredicate(Literal(2) > decimalAttr, classOf[Lt[_]], resultFun(1)) + checkFilterPredicate(Literal(3) < decimalAttr, classOf[Gt[_]], resultFun(4)) + checkFilterPredicate(Literal(1) >= decimalAttr, classOf[LtEq[_]], resultFun(1)) + checkFilterPredicate(Literal(4) <= decimalAttr, classOf[GtEq[_]], resultFun(4)) + + checkFilterPredicate(!(decimalAttr < 4), classOf[GtEq[_]], resultFun(4)) + checkFilterPredicate(decimalAttr < 2 || decimalAttr > 3, classOf[Operators.Or], + Seq(Row(resultFun(1)), Row(resultFun(4)))) + } } } } @@ -1042,7 +1163,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } test("SPARK-16371 Do not push down filters when inner name and outer name are the same") { - withParquetDataFrame((1 to 4).map(i => Tuple1(Tuple1(i)))) { implicit df => + import testImplicits._ + withParquetDataFrame((1 to 4).map(i => Tuple1(Tuple1(i))).toDF()) { implicit df => // Here the schema becomes as below: // // root @@ -1107,7 +1229,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } - test("SPARK-20364: Disable Parquet predicate pushdown for fields having dots in the names") { + test("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") { import testImplicits._ Seq(true, false).foreach { vectorized => @@ -1120,6 +1242,28 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared assert(readBack.count() == 1) } } + + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized.toString, + // Makes sure disabling 'spark.sql.parquet.recordFilter' still enables + // row group level filtering. + SQLConf.PARQUET_RECORD_FILTER_ENABLED.key -> "false", + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + + withTempPath { path => + val data = (1 to 1024) + data.toDF("col.dots").coalesce(1) + .write.option("parquet.block.size", 512) + .parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath).filter("`col.dots` == 500") + // Here, we strip the Spark side filter and check the actual results from Parquet. + val actual = stripSparkFilter(df).collect().length + // Since those are filtered at row group level, the result count should be less + // than the total length but should not be a single record. + // Note that, if record level filtering is enabled, it should be a single record. + // If no filter is pushed down to Parquet, it should be the total length of data. + assert(actual > 1 && actual < data.length) + } + } } } @@ -1162,7 +1306,10 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } test("filter pushdown - StringStartsWith") { - withParquetDataFrame((1 to 4).map(i => Tuple1(i + "str" + i))) { implicit df => + withParquetDataFrame { + import testImplicits._ + (1 to 4).map(i => Tuple1(i + "str" + i)).toDF() + } { implicit df => checkFilterPredicate( '_1.startsWith("").asInstanceOf[Predicate], classOf[UserDefinedByInstance[_, _]], @@ -1208,7 +1355,10 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } // SPARK-28371: make sure filter is null-safe. - withParquetDataFrame(Seq(Tuple1[String](null))) { implicit df => + withParquetDataFrame { + import testImplicits._ + Seq(Tuple1[String](null)).toDF() + } { implicit df => checkFilterPredicate( '_1.startsWith("blah").asInstanceOf[Predicate], classOf[UserDefinedByInstance[_, _]], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 1550b3bbb6242..f901ce16f0965 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet +import java.sql.{Date, Timestamp} import java.util.Locale import scala.collection.JavaConverters._ @@ -81,7 +82,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession * Writes `data` to a Parquet file, reads it back and check file contents. */ protected def checkParquetFile[T <: Product : ClassTag: TypeTag](data: Seq[T]): Unit = { - withParquetDataFrame(data)(r => checkAnswer(r, data.map(Row.fromTuple))) + withParquetDataFrame(data.toDF())(r => checkAnswer(r, data.map(Row.fromTuple))) } test("basic data types (without binary)") { @@ -93,7 +94,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession test("raw binary") { val data = (1 to 4).map(i => Tuple1(Array.fill(3)(i.toByte))) - withParquetDataFrame(data) { df => + withParquetDataFrame(data.toDF()) { df => assertResult(data.map(_._1.mkString(",")).sorted) { df.collect().map(_.getAs[Array[Byte]](0).mkString(",")).sorted } @@ -196,7 +197,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession testStandardAndLegacyModes("struct") { val data = (1 to 4).map(i => Tuple1((i, s"val_$i"))) - withParquetDataFrame(data) { df => + withParquetDataFrame(data.toDF()) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(struct) => Row(Row(struct.productIterator.toSeq: _*)) @@ -213,7 +214,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession ) ) } - withParquetDataFrame(data) { df => + withParquetDataFrame(data.toDF()) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(array) => Row(array.map(struct => Row(struct.productIterator.toSeq: _*))) @@ -232,7 +233,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession ) ) } - withParquetDataFrame(data) { df => + withParquetDataFrame(data.toDF()) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(array) => Row(array.map { case Tuple1(Tuple1(str)) => Row(Row(str))}) @@ -242,7 +243,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession testStandardAndLegacyModes("nested struct with array of array as field") { val data = (1 to 4).map(i => Tuple1((i, Seq(Seq(s"val_$i"))))) - withParquetDataFrame(data) { df => + withParquetDataFrame(data.toDF()) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(struct) => Row(Row(struct.productIterator.toSeq: _*)) @@ -259,7 +260,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession ) ) } - withParquetDataFrame(data) { df => + withParquetDataFrame(data.toDF()) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(m) => Row(m.map { case (k, v) => Row(k.productIterator.toSeq: _*) -> v }) @@ -276,7 +277,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession ) ) } - withParquetDataFrame(data) { df => + withParquetDataFrame(data.toDF()) { df => // Structs are converted to `Row`s checkAnswer(df, data.map { case Tuple1(m) => Row(m.mapValues(struct => Row(struct.productIterator.toSeq: _*))) @@ -292,7 +293,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession null.asInstanceOf[java.lang.Float], null.asInstanceOf[java.lang.Double]) - withParquetDataFrame(allNulls :: Nil) { df => + withParquetDataFrame((allNulls :: Nil).toDF()) { df => val rows = df.collect() assert(rows.length === 1) assert(rows.head === Row(Seq.fill(5)(null): _*)) @@ -305,7 +306,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession None.asInstanceOf[Option[Long]], None.asInstanceOf[Option[String]]) - withParquetDataFrame(allNones :: Nil) { df => + withParquetDataFrame((allNones :: Nil).toDF()) { df => val rows = df.collect() assert(rows.length === 1) assert(rows.head === Row(Seq.fill(3)(null): _*)) @@ -879,6 +880,73 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession assert(metaData.get(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) } } + + test("SPARK-31159: compatibility with Spark 2.4 in reading dates/timestamps") { + Seq(false, true).foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_READ.key -> "true") { + checkAnswer( + readResourceParquetFile("test-data/before_1582_date_v2_4.snappy.parquet"), + Row(java.sql.Date.valueOf("1001-01-01"))) + checkAnswer(readResourceParquetFile( + "test-data/before_1582_timestamp_micros_v2_4.snappy.parquet"), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"))) + checkAnswer(readResourceParquetFile( + "test-data/before_1582_timestamp_millis_v2_4.snappy.parquet"), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123"))) + } + checkAnswer(readResourceParquetFile( + "test-data/before_1582_timestamp_int96_v2_4.snappy.parquet"), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"))) + } + } + } + + test("SPARK-31159: rebasing timestamps in write") { + Seq( + ("TIMESTAMP_MILLIS", "1001-01-01 01:02:03.123", "1001-01-07 01:09:05.123"), + ("TIMESTAMP_MICROS", "1001-01-01 01:02:03.123456", "1001-01-07 01:09:05.123456"), + ("INT96", "1001-01-01 01:02:03.123456", "1001-01-01 01:02:03.123456") + ).foreach { case (outType, tsStr, nonRebased) => + withClue(s"output type $outType") { + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> outType) { + withTempPath { dir => + val path = dir.getAbsolutePath + withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_WRITE.key -> "true") { + Seq(tsStr).toDF("tsS") + .select($"tsS".cast("timestamp").as("ts")) + .write + .parquet(path) + } + withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_READ.key -> "true") { + checkAnswer(spark.read.parquet(path), Row(Timestamp.valueOf(tsStr))) + } + withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_READ.key -> "false") { + checkAnswer(spark.read.parquet(path), Row(Timestamp.valueOf(nonRebased))) + } + } + } + } + } + } + + test("SPARK-31159: rebasing dates in write") { + withTempPath { dir => + val path = dir.getAbsolutePath + withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_WRITE.key -> "true") { + Seq("1001-01-01").toDF("dateS") + .select($"dateS".cast("date").as("date")) + .write + .parquet(path) + } + withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_READ.key -> "true") { + checkAnswer(spark.read.parquet(path), Row(Date.valueOf("1001-01-01"))) + } + withSQLConf(SQLConf.LEGACY_PARQUET_REBASE_DATETIME_IN_READ.key -> "false") { + checkAnswer(spark.read.parquet(path), Row(Date.valueOf("1001-01-07"))) + } + } + } } class JobCommitFailureParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index e63929470ce5f..d3301ced2ba19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -59,7 +59,8 @@ abstract class ParquetPartitionDiscoverySuite val timeZoneId = ZoneId.systemDefault() val df = DateFormatter(timeZoneId) - val tf = TimestampFormatter(timestampPartitionPattern, timeZoneId) + val tf = TimestampFormatter( + timestampPartitionPattern, timeZoneId, needVarLengthSecondFraction = true) protected override def beforeAll(): Unit = { super.beforeAll() @@ -87,7 +88,7 @@ abstract class ParquetPartitionDiscoverySuite check("1990-02-24 12:00:30", Literal.create(Timestamp.valueOf("1990-02-24 12:00:30"), TimestampType)) - val c = Calendar.getInstance(TimeZone.getTimeZone("GMT")) + val c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) c.set(1990, 1, 24, 12, 0, 30) c.set(Calendar.MILLISECOND, 0) check("1990-02-24 12:00:30", @@ -691,10 +692,10 @@ abstract class ParquetPartitionDiscoverySuite } withTempPath { dir => - df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + df.write.option(DateTimeUtils.TIMEZONE_OPTION, "UTC") .format("parquet").partitionBy(partitionColumns.map(_.name): _*).save(dir.toString) val fields = schema.map(f => Column(f.name).cast(f.dataType)) - checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "UTC") .load(dir.toString).select(fields: _*), row) } } @@ -733,10 +734,10 @@ abstract class ParquetPartitionDiscoverySuite } withTempPath { dir => - df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + df.write.option(DateTimeUtils.TIMEZONE_OPTION, "UTC") .format("parquet").partitionBy(partitionColumns.map(_.name): _*).save(dir.toString) val fields = schema.map(f => Column(f.name)) - checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "UTC") .load(dir.toString).select(fields: _*), row) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index 828ba6aee026b..f2dbc536ac566 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -63,12 +63,16 @@ private[sql] trait ParquetTest extends FileBasedDataSourceTest { (f: String => Unit): Unit = withDataSourceFile(data)(f) /** - * Writes `data` to a Parquet file and reads it back as a [[DataFrame]], + * Writes `df` dataframe to a Parquet file and reads it back as a [[DataFrame]], * which is then passed to `f`. The Parquet file will be deleted after `f` returns. */ - protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag] - (data: Seq[T], testVectorized: Boolean = true) - (f: DataFrame => Unit): Unit = withDataSourceDataFrame(data, testVectorized)(f) + protected def withParquetDataFrame(df: DataFrame, testVectorized: Boolean = true) + (f: DataFrame => Unit): Unit = { + withTempPath { file => + df.write.format(dataSourceName).save(file.getCanonicalPath) + readFile(file.getCanonicalPath, testVectorized)(f) + } + } /** * Writes `data` to a Parquet file, reads it back as a [[DataFrame]] and registers it as a diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index 4cb845b2487d6..e9ef7c1a0c540 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -24,27 +24,14 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.StructType -class DebuggingSuite extends SharedSparkSession { - - - var originalValue: String = _ - // With on AQE, the WholeStageCodegenExec is added when running QueryStageExec. - override def beforeAll(): Unit = { - super.beforeAll() - originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") - } - - override def afterAll(): Unit = { - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) - super.afterAll() - } +// Disable AQE because the WholeStageCodegenExec is added when running QueryStageExec +class DebuggingSuite extends SharedSparkSession with DisableAdaptiveExecutionSuite { test("DataFrame.debug()") { testData.debug() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 7d09577075d5d..1c4a8189349ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.functions._ @@ -33,22 +34,11 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.{AccumulatorContext, JsonProtocol} -class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { +// Disable AQE because metric info is different with AQE on/off +class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils + with DisableAdaptiveExecutionSuite { import testImplicits._ - var originalValue: String = _ - // With AQE on/off, the metric info is different. - override def beforeAll(): Unit = { - super.beforeAll() - originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") - } - - override def afterAll(): Unit = { - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) - super.afterAll() - } - /** * Generates a `DataFrame` by filling randomly generated bytes for hash collision. */ @@ -98,7 +88,7 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { val ds = spark.range(10).filter('id < 5) testSparkPlanMetricsWithPredicates(ds.toDF(), 1, Map( 0L -> (("WholeStageCodegen (1)", Map( - "duration total (min, med, max (stageId (attemptId): taskId))" -> { + "duration" -> { _.toString.matches(timingMetricPattern) })))), true) } @@ -110,10 +100,10 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { val df = testData2.groupBy().count() // 2 partitions val expected1 = Seq( Map("number of output rows" -> 2L, - "avg hash probe bucket list iters (min, med, max (stageId (attemptId): taskId))" -> + "avg hash probe bucket list iters" -> aggregateMetricsPattern), Map("number of output rows" -> 1L, - "avg hash probe bucket list iters (min, med, max (stageId (attemptId): taskId))" -> + "avg hash probe bucket list iters" -> aggregateMetricsPattern)) val shuffleExpected1 = Map( "records read" -> 2L, @@ -130,10 +120,10 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { val df2 = testData2.groupBy('a).count() val expected2 = Seq( Map("number of output rows" -> 4L, - "avg hash probe bucket list iters (min, med, max (stageId (attemptId): taskId))" -> + "avg hash probe bucket list iters" -> aggregateMetricsPattern), Map("number of output rows" -> 3L, - "avg hash probe bucket list iters (min, med, max (stageId (attemptId): taskId))" -> + "avg hash probe bucket list iters" -> aggregateMetricsPattern)) val shuffleExpected2 = Map( @@ -181,12 +171,17 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { } val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe bucket list iters (min, med, max (stageId" + - " (attemptId): taskId))") - // Extract min, med, max from the string and strip off everthing else. - val index = probes.toString.stripPrefix("\n(").stripSuffix(")").indexOf(" (", 0) - probes.toString.stripPrefix("\n(").stripSuffix(")").slice(0, index).split(", ").foreach { - probe => assert(probe.toDouble > 1.0) + val probes = metrics(nodeId)._2("avg hash probe bucket list iters").toString + if (!probes.contains("\n")) { + // It's a single metrics value + assert(probes.toDouble > 1.0) + } else { + val mainValue = probes.split("\n").apply(1).stripPrefix("(").stripSuffix(")") + // Extract min, med, max from the string and strip off everthing else. + val index = mainValue.indexOf(" (", 0) + mainValue.slice(0, index).split(", ").foreach { + probe => assert(probe.toDouble > 1.0) + } } } } @@ -231,13 +226,13 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { val df = Seq(1, 3, 2).toDF("id").sort('id) testSparkPlanMetricsWithPredicates(df, 2, Map( 0L -> (("Sort", Map( - "sort time total (min, med, max (stageId (attemptId): taskId))" -> { + "sort time" -> { _.toString.matches(timingMetricPattern) }, - "peak memory total (min, med, max (stageId (attemptId): taskId))" -> { + "peak memory" -> { _.toString.matches(sizeMetricPattern) }, - "spill size total (min, med, max (stageId (attemptId): taskId))" -> { + "spill size" -> { _.toString.matches(sizeMetricPattern) }))) )) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala index 0c1148f7b82e4..2977b5339ab51 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -41,28 +41,28 @@ trait SQLMetricsTestUtils extends SQLTestUtils { protected def statusStore: SQLAppStatusStore = spark.sharedState.statusStore - // Pattern of size SQLMetric value, e.g. "\n96.2 MiB (32.1 MiB, 32.1 MiB, 32.1 MiB (stage 0 - // (attempt 0): task 4))" OR "\n96.2 MiB (32.1 MiB, 32.1 MiB, 32.1 MiB)" + // Pattern of size SQLMetric value, e.g. "\n96.2 MiB (32.1 MiB, 32.1 MiB, 32.1 MiB (stage 0.0: + // task 4))" OR "\n96.2 MiB (32.1 MiB, 32.1 MiB, 32.1 MiB)" protected val sizeMetricPattern = { val bytes = "([0-9]+(\\.[0-9]+)?) (EiB|PiB|TiB|GiB|MiB|KiB|B)" - val maxMetrics = "\\(stage ([0-9])+ \\(attempt ([0-9])+\\)\\: task ([0-9])+\\)" - s"\\n$bytes \\($bytes, $bytes, $bytes( $maxMetrics)?\\)" + val maxMetrics = "\\(stage ([0-9])+\\.([0-9])+\\: task ([0-9])+\\)" + s"(.*\\n$bytes \\($bytes, $bytes, $bytes( $maxMetrics)?\\))|($bytes)" } - // Pattern of timing SQLMetric value, e.g. "\n2.0 ms (1.0 ms, 1.0 ms, 1.0 ms (stage 3 (attempt - // 0): task 217))" OR "\n2.0 ms (1.0 ms, 1.0 ms, 1.0 ms)" + // Pattern of timing SQLMetric value, e.g. "\n2.0 ms (1.0 ms, 1.0 ms, 1.0 ms (stage 3.0): + // task 217))" OR "\n2.0 ms (1.0 ms, 1.0 ms, 1.0 ms)" OR "1.0 ms" protected val timingMetricPattern = { val duration = "([0-9]+(\\.[0-9]+)?) (ms|s|m|h)" - val maxMetrics = "\\(stage ([0-9])+ \\(attempt ([0-9])+\\)\\: task ([0-9])+\\)" - s"\\n$duration \\($duration, $duration, $duration( $maxMetrics)?\\)" + val maxMetrics = "\\(stage ([0-9])+\\.([0-9])+\\: task ([0-9])+\\)" + s"(.*\\n$duration \\($duration, $duration, $duration( $maxMetrics)?\\))|($duration)" } // Pattern of size SQLMetric value for Aggregate tests. - // e.g "\n(1, 1, 0.9 (stage 1 (attempt 0): task 8)) OR "\n(1, 1, 0.9 )" + // e.g "\n(1, 1, 0.9 (stage 1.0: task 8)) OR "\n(1, 1, 0.9 )" OR "1" protected val aggregateMetricsPattern = { val iters = "([0-9]+(\\.[0-9]+)?)" - val maxMetrics = "\\(stage ([0-9])+ \\(attempt ([0-9])+\\)\\: task ([0-9])+\\)" - s"\\n\\($iters, $iters, $iters( $maxMetrics)?\\)" + val maxMetrics = "\\(stage ([0-9])+\\.([0-9])+\\: task ([0-9])+\\)" + s"(.*\\n\\($iters, $iters, $iters( $maxMetrics)?\\))|($iters)" } /** @@ -98,7 +98,7 @@ trait SQLMetricsTestUtils extends SQLTestUtils { } val totalNumBytesMetric = executedNode.metrics.find( - _.name == "written output total (min, med, max (stageId (attemptId): taskId))").get + _.name == "written output").get val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "") .split(" ").head.trim.toDouble assert(totalNumBytes > 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index d18a35c3110f9..4d5cd109b7c24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlanInfo, SQLExecution} +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.functions.count import org.apache.spark.sql.internal.SQLConf @@ -154,11 +155,14 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils expected.foreach { case (id, value) => // The values in actual can be SQL metrics meaning that they contain additional formatting // when converted to string. Verify that they start with the expected value. - // TODO: this is brittle. There is no requirement that the actual string needs to start - // with the accumulator value. assert(actual.contains(id)) val v = actual(id).trim - assert(v.startsWith(value.toString), s"Wrong value for accumulator $id") + if (v.contains("\n")) { + // The actual value can be "total (max, ...)\n6 ms (5 ms, ...)". + assert(v.split("\n")(1).startsWith(value.toString), s"Wrong value for accumulator $id") + } else { + assert(v.startsWith(value.toString), s"Wrong value for accumulator $id") + } } } @@ -314,6 +318,43 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 11)) } + test("control a plan explain mode in listeners via SQLConf") { + + def checkPlanDescription(mode: String, expected: Seq[String]): Unit = { + var checkDone = false + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case SparkListenerSQLExecutionStart(_, _, _, planDescription, _, _) => + assert(expected.forall(planDescription.contains)) + checkDone = true + case _ => // ignore other events + } + } + } + spark.sparkContext.addSparkListener(listener) + withSQLConf(SQLConf.UI_EXPLAIN_MODE.key -> mode) { + createTestDataFrame.collect() + try { + spark.sparkContext.listenerBus.waitUntilEmpty() + assert(checkDone) + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } + } + + Seq(("simple", Seq("== Physical Plan ==")), + ("extended", Seq("== Parsed Logical Plan ==", "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==", "== Physical Plan ==")), + ("codegen", Seq("WholeStageCodegen subtrees")), + ("cost", Seq("== Optimized Logical Plan ==", "Statistics(sizeInBytes")), + ("formatted", Seq("== Physical Plan ==", "Output", "Arguments"))).foreach { + case (mode, expected) => + checkPlanDescription(mode, expected) + } + } + test("onExecutionEnd happens before onJobEnd(JobSucceeded)") { val statusStore = createStatusStore() val listener = statusStore.listener.get @@ -622,13 +663,12 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils assert(statusStore.execution(2) === None) } - test("SPARK-29894 test Codegen Stage Id in SparkPlanInfo") { - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - // with AQE on, the WholeStageCodegen rule is applied when running QueryStageExec. - val df = createTestDataFrame.select(count("*")) - val sparkPlanInfo = SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan) - assert(sparkPlanInfo.nodeName === "WholeStageCodegen (2)") - } + test("SPARK-29894 test Codegen Stage Id in SparkPlanInfo", + DisableAdaptiveExecution("WSCG rule is applied later in AQE")) { + // with AQE on, the WholeStageCodegen rule is applied when running QueryStageExec. + val df = createTestDataFrame.select(count("*")) + val sparkPlanInfo = SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan) + assert(sparkPlanInfo.nodeName === "WholeStageCodegen (2)") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/DeprecatedCreateExternalTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/DeprecatedCreateExternalTableSuite.scala new file mode 100644 index 0000000000000..0b5cd3dae4761 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/DeprecatedCreateExternalTableSuite.scala @@ -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.sql.internal + +import java.io.File + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.StructType + +class DeprecatedCreateExternalTableSuite extends SharedSparkSession { + test("createExternalTable with explicit path") { + withTable("t") { + withTempDir { dir => + val path = new File(dir, "test") + spark.range(100).write.parquet(path.getAbsolutePath) + spark.catalog.createExternalTable( + tableName = "t", + path = path.getAbsolutePath + ) + assert(spark.sessionState.catalog.tableExists(TableIdentifier("t"))) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.tableType === CatalogTableType.EXTERNAL) + assert(table.provider === Some("parquet")) + assert(table.schema === new StructType().add("id", "long")) + assert(table.storage.locationUri.get == makeQualifiedPath(path.getAbsolutePath)) + } + } + } + + test("createExternalTable with 'path' options") { + withTable("t") { + withTempDir { dir => + val path = new File(dir, "test") + spark.range(100).write.parquet(path.getAbsolutePath) + spark.catalog.createExternalTable( + tableName = "t", + source = "parquet", + options = Map("path" -> path.getAbsolutePath)) + assert(spark.sessionState.catalog.tableExists(TableIdentifier("t"))) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.tableType === CatalogTableType.EXTERNAL) + assert(table.provider === Some("parquet")) + assert(table.schema === new StructType().add("id", "long")) + assert(table.storage.locationUri.get == makeQualifiedPath(path.getAbsolutePath)) + } + } + } + + test("createExternalTable with explicit schema") { + withTable("t") { + withTempDir { dir => + val path = new File(dir, "test") + spark.range(100).write.parquet(path.getAbsolutePath) + spark.catalog.createExternalTable( + tableName = "t", + source = "parquet", + schema = new StructType().add("i", "int"), + options = Map("path" -> path.getAbsolutePath)) + assert(spark.sessionState.catalog.tableExists(TableIdentifier("t"))) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.tableType === CatalogTableType.EXTERNAL) + assert(table.provider === Some("parquet")) + assert(table.schema === new StructType().add("i", "int")) + assert(table.storage.locationUri.get == makeQualifiedPath(path.getAbsolutePath)) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index 888772c35d0ee..567524ac75c2e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlan} +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.debug.codegenStringSeq import org.apache.spark.sql.functions.col import org.apache.spark.sql.test.SQLTestUtils @@ -98,10 +99,10 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { } } - test("SPARK-22219: refactor to control to generate comment") { + test("SPARK-22219: refactor to control to generate comment", + DisableAdaptiveExecution("WSCG rule is applied later in AQE")) { Seq(true, false).foreach { flag => - withSQLConf(StaticSQLConf.CODEGEN_COMMENTS.key -> flag.toString, - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + withSQLConf(StaticSQLConf.CODEGEN_COMMENTS.key -> flag.toString) { // with AQE on, the WholeStageCodegen rule is applied when running QueryStageExec. val res = codegenStringSeq(spark.range(10).groupBy(col("id") * 2).count() .queryExecution.executedPlan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 650dc127a060a..f389465d4b489 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.log4j.Level import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.MIT import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext} import org.apache.spark.util.Utils @@ -115,6 +116,21 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } } + test("reset will not change static sql configs and spark core configs") { + val conf = spark.sparkContext.getConf.getAll.toMap + val appName = conf.get("spark.app.name") + val driverHost = conf.get("spark.driver.host") + val master = conf.get("spark.master") + val warehouseDir = conf.get("spark.sql.warehouse.dir") + // ensure the conf here is not default value, and will not be reset to default value later + assert(warehouseDir.get.contains(this.getClass.getCanonicalName)) + sql("RESET") + assert(conf.get("spark.app.name") === appName) + assert(conf.get("spark.driver.host") === driverHost) + assert(conf.get("spark.master") === master) + assert(conf.get("spark.sql.warehouse.dir") === warehouseDir) + } + test("reset - public conf") { spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) @@ -350,8 +366,8 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } test("spark.sql.session.timeZone should only accept valid zone id") { - spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "MIT") - assert(sql(s"set ${SQLConf.SESSION_LOCAL_TIMEZONE.key}").head().getString(1) === "MIT") + spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, MIT.getId) + assert(sql(s"set ${SQLConf.SESSION_LOCAL_TIMEZONE.key}").head().getString(1) === MIT.getId) spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "America/Chicago") assert(sql(s"set ${SQLConf.SESSION_LOCAL_TIMEZONE.key}").head().getString(1) === "America/Chicago") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 9cba95f7d7df2..fd691f547eb05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -1699,4 +1699,18 @@ class JDBCSuite extends QueryTest assert(JdbcDialects.get("jdbc:teradata://localhost/db") === TeradataDialect) assert(JdbcDialects.get("jdbc:Teradata://localhost/db") === TeradataDialect) } + + test("SQLContext.jdbc (deprecated)") { + val sqlContext = spark.sqlContext + var jdbcDF = sqlContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE") + checkAnswer(jdbcDF, Row("fred", 1) :: Row("mary", 2) :: Row ("joe 'foo' \"bar\"", 3) :: Nil) + + jdbcDF = sqlContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", "THEID", 0, 4, 3) + checkNumPartitions(jdbcDF, 3) + checkAnswer(jdbcDF, Row("fred", 1) :: Row("mary", 2) :: Row ("joe 'foo' \"bar\"", 3) :: Nil) + + val parts = Array[String]("THEID = 2") + jdbcDF = sqlContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts) + checkAnswer(jdbcDF, Row("mary", 2) :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 7242278b59eff..e153c7168dbf2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -27,8 +27,9 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{DataSourceScanExec, SortExec} +import org.apache.spark.sql.execution.{DataSourceScanExec, FileSourceScanExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.execution.adaptive.AdaptiveTestUtils.assertExceptionMessage import org.apache.spark.sql.execution.datasources.BucketingUtils import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec @@ -100,6 +101,12 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } + private def getFileScan(plan: SparkPlan): FileSourceScanExec = { + val fileScan = plan.collect { case f: FileSourceScanExec => f } + assert(fileScan.nonEmpty, plan) + fileScan.head + } + // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one @@ -119,8 +126,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { // Filter could hide the bug in bucket pruning. Thus, skipping all the filters val plan = bucketedDataFrame.filter(filterCondition).queryExecution.executedPlan - val rdd = plan.find(_.isInstanceOf[DataSourceScanExec]) - assert(rdd.isDefined, plan) + val fileScan = getFileScan(plan) // if nothing should be pruned, skip the pruning test if (bucketValues.nonEmpty) { @@ -128,7 +134,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { bucketValues.foreach { value => matchedBuckets.set(BucketingUtils.getBucketIdFromValue(bucketColumn, numBuckets, value)) } - val invalidBuckets = rdd.get.execute().mapPartitionsWithIndex { case (index, iter) => + val invalidBuckets = fileScan.execute().mapPartitionsWithIndex { case (index, iter) => // return indexes of partitions that should have been pruned and are not empty if (!matchedBuckets.get(index % numBuckets) && iter.nonEmpty) { Iterator(index) @@ -297,10 +303,9 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { val bucketedDataFrame = spark.table("bucketed_table").select("i", "j", "k") val plan = bucketedDataFrame.queryExecution.executedPlan - val rdd = plan.find(_.isInstanceOf[DataSourceScanExec]) - assert(rdd.isDefined, plan) + val fileScan = getFileScan(plan) - val emptyBuckets = rdd.get.execute().mapPartitionsWithIndex { case (index, iter) => + val emptyBuckets = fileScan.execute().mapPartitionsWithIndex { case (index, iter) => // return indexes of empty partitions if (iter.isEmpty) { Iterator(index) @@ -766,7 +771,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { agged.count() } - assert(error.getCause().toString contains "Invalid bucket file") + assertExceptionMessage(error, "Invalid bucket file") } } @@ -774,10 +779,13 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") - checkAnswer(spark.table("bucketed_table").select("j"), df1.select("j")) + val scanDF = spark.table("bucketed_table").select("j") + assert(!getFileScan(scanDF.queryExecution.executedPlan).bucketedScan) + checkAnswer(scanDF, df1.select("j")) - checkAnswer(spark.table("bucketed_table").groupBy("j").agg(max("k")), - df1.groupBy("j").agg(max("k"))) + val aggDF = spark.table("bucketed_table").groupBy("j").agg(max("k")) + assert(!getFileScan(aggDF.queryExecution.executedPlan).bucketedScan) + checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FiltersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FiltersSuite.scala index 1cb7a2156c3d3..33b2db57d9f0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FiltersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FiltersSuite.scala @@ -24,66 +24,143 @@ import org.apache.spark.SparkFunSuite */ class FiltersSuite extends SparkFunSuite { - test("EqualTo references") { - assert(EqualTo("a", "1").references.toSeq == Seq("a")) - assert(EqualTo("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) + private def withFieldNames(f: (String, Array[String]) => Unit): Unit = { + Seq(("a", Array("a")), + ("a.b", Array("a", "b")), + ("`a.b`.c", Array("a.b", "c")), + ("`a.b`.`c.d`.`e.f`", Array("a.b", "c.d", "e.f")) + ).foreach { case (name, fieldNames) => + f(name, fieldNames) + } } - test("EqualNullSafe references") { - assert(EqualNullSafe("a", "1").references.toSeq == Seq("a")) - assert(EqualNullSafe("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) - } + test("EqualTo references") { withFieldNames { (name, fieldNames) => + assert(EqualTo(name, "1").references.toSeq == Seq(name)) + assert(EqualTo(name, "1").v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) - test("GreaterThan references") { - assert(GreaterThan("a", "1").references.toSeq == Seq("a")) - assert(GreaterThan("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) - } + assert(EqualTo(name, EqualTo("b", "2")).references.toSeq == Seq(name, "b")) + assert(EqualTo("b", EqualTo(name, "2")).references.toSeq == Seq("b", name)) - test("GreaterThanOrEqual references") { - assert(GreaterThanOrEqual("a", "1").references.toSeq == Seq("a")) - assert(GreaterThanOrEqual("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) - } + assert(EqualTo(name, EqualTo("b", "2")).v2references.toSeq.map(_.toSeq) + == Seq(fieldNames.toSeq, Seq("b"))) + assert(EqualTo("b", EqualTo(name, "2")).v2references.toSeq.map(_.toSeq) + == Seq(Seq("b"), fieldNames.toSeq)) + }} - test("LessThan references") { - assert(LessThan("a", "1").references.toSeq == Seq("a")) - assert(LessThan("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) - } + test("EqualNullSafe references") { withFieldNames { (name, fieldNames) => + assert(EqualNullSafe(name, "1").references.toSeq == Seq(name)) + assert(EqualNullSafe(name, "1").v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) - test("LessThanOrEqual references") { - assert(LessThanOrEqual("a", "1").references.toSeq == Seq("a")) - assert(LessThanOrEqual("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) - } + assert(EqualNullSafe(name, EqualTo("b", "2")).references.toSeq == Seq(name, "b")) + assert(EqualNullSafe("b", EqualTo(name, "2")).references.toSeq == Seq("b", name)) - test("In references") { - assert(In("a", Array("1")).references.toSeq == Seq("a")) - assert(In("a", Array("1", EqualTo("b", "2"))).references.toSeq == Seq("a", "b")) - } + assert(EqualNullSafe(name, EqualTo("b", "2")).v2references.toSeq.map(_.toSeq) + == Seq(fieldNames.toSeq, Seq("b"))) + assert(EqualNullSafe("b", EqualTo(name, "2")).v2references.toSeq.map(_.toSeq) + == Seq(Seq("b"), fieldNames.toSeq)) + }} - test("IsNull references") { - assert(IsNull("a").references.toSeq == Seq("a")) - } + test("GreaterThan references") { withFieldNames { (name, fieldNames) => + assert(GreaterThan(name, "1").references.toSeq == Seq(name)) + assert(GreaterThan(name, "1").v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) - test("IsNotNull references") { - assert(IsNotNull("a").references.toSeq == Seq("a")) - } + assert(GreaterThan(name, EqualTo("b", "2")).references.toSeq == Seq(name, "b")) + assert(GreaterThan("b", EqualTo(name, "2")).references.toSeq == Seq("b", name)) - test("And references") { - assert(And(EqualTo("a", "1"), EqualTo("b", "1")).references.toSeq == Seq("a", "b")) - } + assert(GreaterThan(name, EqualTo("b", "2")).v2references.toSeq.map(_.toSeq) + == Seq(fieldNames.toSeq, Seq("b"))) + assert(GreaterThan("b", EqualTo(name, "2")).v2references.toSeq.map(_.toSeq) + == Seq(Seq("b"), fieldNames.toSeq)) + }} - test("Or references") { - assert(Or(EqualTo("a", "1"), EqualTo("b", "1")).references.toSeq == Seq("a", "b")) - } + test("GreaterThanOrEqual references") { withFieldNames { (name, fieldNames) => + assert(GreaterThanOrEqual(name, "1").references.toSeq == Seq(name)) + assert(GreaterThanOrEqual(name, "1").v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) - test("StringStartsWith references") { - assert(StringStartsWith("a", "str").references.toSeq == Seq("a")) - } + assert(GreaterThanOrEqual(name, EqualTo("b", "2")).references.toSeq == Seq(name, "b")) + assert(GreaterThanOrEqual("b", EqualTo(name, "2")).references.toSeq == Seq("b", name)) - test("StringEndsWith references") { - assert(StringEndsWith("a", "str").references.toSeq == Seq("a")) - } + assert(GreaterThanOrEqual(name, EqualTo("b", "2")).v2references.toSeq.map(_.toSeq) + == Seq(fieldNames.toSeq, Seq("b"))) + assert(GreaterThanOrEqual("b", EqualTo(name, "2")).v2references.toSeq.map(_.toSeq) + == Seq(Seq("b"), fieldNames.toSeq)) + }} - test("StringContains references") { - assert(StringContains("a", "str").references.toSeq == Seq("a")) - } + test("LessThan references") { withFieldNames { (name, fieldNames) => + assert(LessThan(name, "1").references.toSeq == Seq(name)) + assert(LessThan(name, "1").v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) + + assert(LessThan("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) + }} + + test("LessThanOrEqual references") { withFieldNames { (name, fieldNames) => + assert(LessThanOrEqual(name, "1").references.toSeq == Seq(name)) + assert(LessThanOrEqual(name, "1").v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) + + assert(LessThanOrEqual(name, EqualTo("b", "2")).references.toSeq == Seq(name, "b")) + assert(LessThanOrEqual("b", EqualTo(name, "2")).references.toSeq == Seq("b", name)) + + assert(LessThanOrEqual(name, EqualTo("b", "2")).v2references.toSeq.map(_.toSeq) + == Seq(fieldNames.toSeq, Seq("b"))) + assert(LessThanOrEqual("b", EqualTo(name, "2")).v2references.toSeq.map(_.toSeq) + == Seq(Seq("b"), fieldNames.toSeq)) + }} + + test("In references") { withFieldNames { (name, fieldNames) => + assert(In(name, Array("1")).references.toSeq == Seq(name)) + assert(In(name, Array("1")).v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) + + assert(In(name, Array("1", EqualTo("b", "2"))).references.toSeq == Seq(name, "b")) + assert(In("b", Array("1", EqualTo(name, "2"))).references.toSeq == Seq("b", name)) + + assert(In(name, Array("1", EqualTo("b", "2"))).v2references.toSeq.map(_.toSeq) + == Seq(fieldNames.toSeq, Seq("b"))) + assert(In("b", Array("1", EqualTo(name, "2"))).v2references.toSeq.map(_.toSeq) + == Seq(Seq("b"), fieldNames.toSeq)) + }} + + test("IsNull references") { withFieldNames { (name, fieldNames) => + assert(IsNull(name).references.toSeq == Seq(name)) + assert(IsNull(name).v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) + }} + + test("IsNotNull references") { withFieldNames { (name, fieldNames) => + assert(IsNotNull(name).references.toSeq == Seq(name)) + assert(IsNull(name).v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) + }} + + test("And references") { withFieldNames { (name, fieldNames) => + assert(And(EqualTo(name, "1"), EqualTo("b", "1")).references.toSeq == Seq(name, "b")) + assert(And(EqualTo("b", "1"), EqualTo(name, "1")).references.toSeq == Seq("b", name)) + + assert(And(EqualTo(name, "1"), EqualTo("b", "1")).v2references.toSeq.map(_.toSeq) == + Seq(fieldNames.toSeq, Seq("b"))) + assert(And(EqualTo("b", "1"), EqualTo(name, "1")).v2references.toSeq.map(_.toSeq) == + Seq(Seq("b"), fieldNames.toSeq)) + }} + + test("Or references") { withFieldNames { (name, fieldNames) => + assert(Or(EqualTo(name, "1"), EqualTo("b", "1")).references.toSeq == Seq(name, "b")) + assert(Or(EqualTo("b", "1"), EqualTo(name, "1")).references.toSeq == Seq("b", name)) + + assert(Or(EqualTo(name, "1"), EqualTo("b", "1")).v2references.toSeq.map(_.toSeq) == + Seq(fieldNames.toSeq, Seq("b"))) + assert(Or(EqualTo("b", "1"), EqualTo(name, "1")).v2references.toSeq.map(_.toSeq) == + Seq(Seq("b"), fieldNames.toSeq)) + }} + + test("StringStartsWith references") { withFieldNames { (name, fieldNames) => + assert(StringStartsWith(name, "str").references.toSeq == Seq(name)) + assert(StringStartsWith(name, "str").v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) + }} + + test("StringEndsWith references") { withFieldNames { (name, fieldNames) => + assert(StringEndsWith(name, "str").references.toSeq == Seq(name)) + assert(StringEndsWith(name, "str").v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) + }} + + test("StringContains references") { withFieldNames { (name, fieldNames) => + assert(StringContains(name, "str").references.toSeq == Seq(name)) + assert(StringContains(name, "str").v2references.toSeq.map(_.toSeq) == Seq(fieldNames.toSeq)) + }} } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index c81080ebe4d61..7c48f046c72a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -846,16 +846,16 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { withTempPath { dir => val path = dir.toURI.getPath - sql(s"""create table tab1 ( a int) location '$path'""") + sql(s"""create table tab1 ( a int) using parquet location '$path'""") sql("insert into tab1 values(1)") checkAnswer(sql("select * from tab1"), Seq(1).map(i => Row(i))) - sql("create table tab2 ( a int)") + sql("create table tab2 ( a int) using parquet") sql("insert into tab2 values(2)") checkAnswer(sql("select * from tab2"), Seq(2).map(i => Row(i))) sql(s"""insert overwrite local directory '$path' using parquet select * from tab2""") sql("refresh table tab1") checkAnswer(sql("select * from tab1"), Seq(2).map(i => Row(i))) - } + } } test("SPARK-29174 fail LOCAL in INSERT OVERWRITE DIRECT remote path") { 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 ab1d1f80e7397..1c4e2a967b0a2 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 @@ -139,15 +139,15 @@ class PartitionedWriteSuite extends QueryTest with SharedSparkSession { checkPartitionValues(files.head, "2016-12-01 00:00:00") } withTempPath { f => - df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + df.write.option(DateTimeUtils.TIMEZONE_OPTION, "UTC") .partitionBy("ts").parquet(f.getAbsolutePath) val files = TestUtils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) assert(files.length == 1) - // use timeZone option "GMT" to format partition value. + // use timeZone option utcTz.getId to format partition value. checkPartitionValues(files.head, "2016-12-01 08:00:00") } withTempPath { f => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { df.write.partitionBy("ts").parquet(f.getAbsolutePath) val files = TestUtils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) assert(files.length == 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 92ec2a0c172ef..f29a6c7f77079 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.{BeforeAndAfter, Matchers} import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, Dataset} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.functions.{count, window} @@ -773,7 +774,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche } private val timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 - timestampFormat.setTimeZone(ju.TimeZone.getTimeZone("UTC")) + timestampFormat.setTimeZone(ju.TimeZone.getTimeZone(UTC)) private def formatTimestamp(sec: Long): String = { timestampFormat.format(new ju.Date(sec * 1000)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala index d36c64f61a726..b04f8b0d4d174 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala @@ -798,7 +798,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { } }, CheckNewAnswer(("c", "-1")), - assertNumStateRows(total = 0, updated = 0) + assertNumStateRows(total = 0, updated = 1) ) } 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 bbbf7e5c4697e..78f769536b24b 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 @@ -1245,9 +1245,10 @@ class StreamSuite extends StreamTest { failAfter(60.seconds) { val startTime = System.nanoTime() withSQLConf(SQLConf.STREAMING_STOP_TIMEOUT.key -> "2000") { - intercept[TimeoutException] { + val ex = intercept[TimeoutException] { sq.stop() } + assert(ex.getMessage.contains(sq.id.toString)) } val duration = (System.nanoTime() - startTime) / 1e6 assert(duration >= 2000, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 741355381222d..85e1b85b84d26 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -202,47 +202,68 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { } } - def stateOperatorProgresses: Seq[StateOperatorProgress] = { - val operatorProgress = mutable.ArrayBuffer[StateOperatorProgress]() - var progress = query.recentProgress.last - - operatorProgress ++= progress.stateOperators.map { op => op.copy(op.numRowsUpdated) } - if (progress.numInputRows == 0) { - // empty batch, merge metrics from previous batch as well - progress = query.recentProgress.takeRight(2).head - operatorProgress.zipWithIndex.foreach { case (sop, index) => - // "numRowsUpdated" should be merged, as it could be updated in both batches. - // (for now it is only updated from previous batch, but things can be changed.) - // other metrics represent current status of state so picking up the latest values. - val newOperatorProgress = sop.copy( - sop.numRowsUpdated + progress.stateOperators(index).numRowsUpdated) - operatorProgress(index) = newOperatorProgress - } - } + // Pick the latest progress that actually ran a batch + def lastExecutedBatch: StreamingQueryProgress = { + query.recentProgress.filter(_.durationMs.containsKey("addBatch")).last + } - operatorProgress + def stateOperatorProgresses: Seq[StateOperatorProgress] = { + lastExecutedBatch.stateOperators } } + val clock = new StreamManualClock() + testStream(aggWithWatermark)( + // batchId 0 AddData(inputData, 15), - CheckAnswer(), // watermark = 5 + StartStream(Trigger.ProcessingTime("interval 1 second"), clock), + CheckAnswer(), // watermark = 0 AssertOnQuery { _.stateNodes.size === 1 }, AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 1 }, AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 1 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 0 }, + + // batchId 1 without data + AdvanceManualClock(1000L), // watermark = 5 + Execute { q => // wait for the no data batch to complete + eventually(timeout(streamingTimeout)) { assert(q.lastProgress.batchId === 1) } + }, + CheckAnswer(), + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 1 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 0 }, + + // batchId 2 with data AddData(inputData, 10, 12, 14), - CheckAnswer(), // watermark = 5 - AssertOnQuery { _.stateNodes.size === 1 }, + AdvanceManualClock(1000L), // watermark = 5 + CheckAnswer(), AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 1 }, AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 2 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 0 }, + + // batchId 3 with data AddData(inputData, 25), - CheckAnswer((10, 3)), // watermark = 15 - AssertOnQuery { _.stateNodes.size === 1 }, - AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 1 }, + AdvanceManualClock(1000L), // watermark = 5 + CheckAnswer(), + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 1 }, - AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 2 } + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 3 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 0 }, + + // batchId 4 without data + AdvanceManualClock(1000L), // watermark = 15 + Execute { q => // wait for the no data batch to complete + eventually(timeout(streamingTimeout)) { assert(q.lastProgress.batchId === 4) } + }, + CheckAnswer((10, 3)), + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 1 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 2 }, + AssertOnQuery { _.lastExecutedBatch.sink.numOutputRows == 1 } ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index cfd7204ea2931..f63778aef5a7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -280,7 +280,8 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { { // State should have been cleaned if flag is set, otherwise should not have been cleaned if (flag) assertNumStateRows(total = 1, updated = 1) else assertNumStateRows(total = 7, updated = 1) - } + }, + AssertOnQuery(q => q.lastProgress.sink.numOutputRows == 0L) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala index 6f00b528cb8bd..08b3644745f9a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -241,6 +241,7 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { assert(nextProgress.numInputRows === 0) assert(nextProgress.stateOperators.head.numRowsTotal === 2) assert(nextProgress.stateOperators.head.numRowsUpdated === 0) + assert(nextProgress.sink.numOutputRows === 0) } } finally { query.stop() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 38893f846e5a4..7be15e9d87004 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -27,7 +27,8 @@ import scala.language.implicitConversions import scala.util.control.NonFatal import org.apache.hadoop.fs.Path -import org.scalatest.{BeforeAndAfterAll, Suite} +import org.scalactic.source.Position +import org.scalatest.{BeforeAndAfterAll, Suite, Tag} import org.scalatest.concurrent.Eventually import org.apache.spark.SparkFunSuite @@ -40,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.FilterExec +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.UninterruptibleThread @@ -114,6 +116,19 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with } } + override protected def test(testName: String, testTags: Tag*)(testFun: => Any) + (implicit pos: Position): Unit = { + if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { + super.test(testName, testTags: _*) { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + testFun + } + } + } else { + super.test(testName, testTags: _*)(testFun) + } + } + /** * Run a test on a separate `UninterruptibleThread`. */ diff --git a/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java index 0dfed768f87de..6601bcb9018f4 100644 --- a/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java +++ b/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java @@ -22,7 +22,9 @@ import org.apache.orc.storage.ql.exec.vector.*; import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.catalyst.util.RebaseDateTime; import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.TimestampType; import org.apache.spark.sql.vectorized.ColumnarArray; @@ -42,6 +44,7 @@ public class OrcColumnVector extends org.apache.spark.sql.vectorized.ColumnVecto private DecimalColumnVector decimalData; private TimestampColumnVector timestampData; private final boolean isTimestamp; + private final boolean isDate; private int batchSize; @@ -54,6 +57,12 @@ public class OrcColumnVector extends org.apache.spark.sql.vectorized.ColumnVecto isTimestamp = false; } + if (type instanceof DateType) { + isDate = true; + } else { + isDate = false; + } + baseData = vector; if (vector instanceof LongColumnVector) { longData = (LongColumnVector) vector; @@ -130,7 +139,12 @@ public short getShort(int rowId) { @Override public int getInt(int rowId) { - return (int) longData.vector[getRowIndex(rowId)]; + int value = (int) longData.vector[getRowIndex(rowId)]; + if (isDate) { + return RebaseDateTime.rebaseJulianToGregorianDays(value); + } else { + return value; + } } @Override diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala new file mode 100644 index 0000000000000..1dccf0ca1faef --- /dev/null +++ b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/DaysWritable.scala @@ -0,0 +1,79 @@ +/* + * 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.orc + +import java.io.{DataInput, DataOutput, IOException} +import java.sql.Date + +import org.apache.hadoop.io.WritableUtils +import org.apache.orc.storage.serde2.io.DateWritable + +import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseJulianToGregorianDays} + +/** + * The class accepts/returns days in Gregorian calendar and rebase them + * via conversion to local date in Julian calendar for dates before 1582-10-15 + * in read/write for backward compatibility with Spark 2.4 and earlier versions. + * + * This is a clone of `org.apache.spark.sql.execution.datasources.DaysWritable`. + * The class is cloned because Hive ORC v1.2 uses different `DateWritable`: + * - v1.2: `org.apache.orc.storage.serde2.io.DateWritable` + * - v2.3 and `HiveInspectors`: `org.apache.hadoop.hive.serde2.io.DateWritable` + * + * @param gregorianDays The number of days since the epoch 1970-01-01 in + * Gregorian calendar. + * @param julianDays The number of days since the epoch 1970-01-01 in + * Julian calendar. + */ +class DaysWritable( + var gregorianDays: Int, + var julianDays: Int) + extends DateWritable { + + def this() = this(0, 0) + def this(gregorianDays: Int) = + this(gregorianDays, rebaseGregorianToJulianDays(gregorianDays)) + def this(dateWritable: DateWritable) = { + this( + gregorianDays = dateWritable match { + case daysWritable: DaysWritable => daysWritable.gregorianDays + case dateWritable: DateWritable => + rebaseJulianToGregorianDays(dateWritable.getDays) + }, + julianDays = dateWritable.getDays) + } + + override def getDays: Int = julianDays + override def get(): Date = new Date(DateWritable.daysToMillis(julianDays)) + + override def set(d: Int): Unit = { + gregorianDays = d + julianDays = rebaseGregorianToJulianDays(d) + } + + @throws[IOException] + override def write(out: DataOutput): Unit = { + WritableUtils.writeVInt(out, julianDays) + } + + @throws[IOException] + override def readFields(in: DataInput): Unit = { + julianDays = WritableUtils.readVInt(in) + gregorianDays = rebaseJulianToGregorianDays(julianDays) + } +} diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index b9cbc484e1fc1..f5abd30854e00 100644 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -65,9 +65,11 @@ private[sql] object OrcFilters extends OrcFiltersBase { * Create ORC filter as a SearchArgument instance. */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + val dataTypeMap = schema.map(f => quoteIfNeeded(f.name) -> f.dataType).toMap // Combines all convertible filters using `And` to produce a single conjunction - val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) + // TODO (SPARK-25557): ORC doesn't support nested predicate pushdown, so they are removed. + val newFilters = filters.filter(!_.containsNestedColumn) + val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, newFilters)) conjunctionOptional.map { conjunction => // Then tries to build a single ORC `SearchArgument` for the conjunction predicate. // The input predicate is fully convertible. There should not be any empty result in the @@ -222,48 +224,39 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Since ORC 1.5.0 (ORC-323), we need to quote for column names with `.` characters // in order to distinguish predicate pushdown for nested columns. expression match { - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().equals(quotedName, getType(attribute), castedValue).end()) + case EqualTo(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startAnd().equals(name, getType(name), castedValue).end()) - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end()) + case EqualNullSafe(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startAnd().nullSafeEquals(name, getType(name), castedValue).end()) - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end()) + case LessThan(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startAnd().lessThan(name, getType(name), castedValue).end()) - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + case LessThanOrEqual(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startAnd().lessThanEquals(name, getType(name), castedValue).end()) - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + case GreaterThan(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startNot().lessThanEquals(name, getType(name), castedValue).end()) - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end()) + case GreaterThanOrEqual(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startNot().lessThan(name, getType(name), castedValue).end()) - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - Some(builder.startAnd().isNull(quotedName, getType(attribute)).end()) + case IsNull(name) if isSearchableType(dataTypeMap(name)) => + Some(builder.startAnd().isNull(name, getType(name)).end()) - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - Some(builder.startNot().isNull(quotedName, getType(attribute)).end()) + case IsNotNull(name) if isSearchableType(dataTypeMap(name)) => + Some(builder.startNot().isNull(name, getType(name)).end()) - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - Some(builder.startAnd().in(quotedName, getType(attribute), + case In(name, values) if isSearchableType(dataTypeMap(name)) => + val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(name))) + Some(builder.startAnd().in(name, getType(name), castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) case _ => None diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala index 68503aba22b40..7fbc1cd205b13 100644 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala +++ b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources.orc -import java.sql.Date - import org.apache.orc.storage.common.`type`.HiveDecimal import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch import org.apache.orc.storage.ql.io.sarg.{SearchArgument => OrcSearchArgument} @@ -38,7 +36,9 @@ private[sql] object OrcShimUtils { private[sql] type Operator = OrcOperator private[sql] type SearchArgument = OrcSearchArgument - def getSqlDate(value: Any): Date = value.asInstanceOf[DateWritable].get + def getGregorianDays(value: Any): Int = { + new DaysWritable(value.asInstanceOf[DateWritable]).gregorianDays + } def getDecimal(value: Any): Decimal = { val decimal = value.asInstanceOf[HiveDecimalWritable].getHiveDecimal() @@ -47,13 +47,13 @@ private[sql] object OrcShimUtils { def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritable = { if (reuseObj) { - val result = new DateWritable() + val result = new DaysWritable() (getter, ordinal) => result.set(getter.getInt(ordinal)) result } else { (getter: SpecializedGetters, ordinal: Int) => - new DateWritable(getter.getInt(ordinal)) + new DaysWritable(getter.getInt(ordinal)) } } diff --git a/sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java index 35447fe7e19c2..6e55fedfc4deb 100644 --- a/sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java +++ b/sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java @@ -22,7 +22,9 @@ import org.apache.hadoop.hive.ql.exec.vector.*; import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.catalyst.util.RebaseDateTime; import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.TimestampType; import org.apache.spark.sql.vectorized.ColumnarArray; @@ -42,6 +44,7 @@ public class OrcColumnVector extends org.apache.spark.sql.vectorized.ColumnVecto private DecimalColumnVector decimalData; private TimestampColumnVector timestampData; private final boolean isTimestamp; + private final boolean isDate; private int batchSize; @@ -54,6 +57,12 @@ public class OrcColumnVector extends org.apache.spark.sql.vectorized.ColumnVecto isTimestamp = false; } + if (type instanceof DateType) { + isDate = true; + } else { + isDate = false; + } + baseData = vector; if (vector instanceof LongColumnVector) { longData = (LongColumnVector) vector; @@ -130,7 +139,12 @@ public short getShort(int rowId) { @Override public int getInt(int rowId) { - return (int) longData.vector[getRowIndex(rowId)]; + int value = (int) longData.vector[getRowIndex(rowId)]; + if (isDate) { + return RebaseDateTime.rebaseJulianToGregorianDays(value); + } else { + return value; + } } @Override diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 6e9e592be13be..675e089153679 100644 --- a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -65,9 +65,11 @@ private[sql] object OrcFilters extends OrcFiltersBase { * Create ORC filter as a SearchArgument instance. */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + val dataTypeMap = schema.map(f => quoteIfNeeded(f.name) -> f.dataType).toMap // Combines all convertible filters using `And` to produce a single conjunction - val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) + // TODO (SPARK-25557): ORC doesn't support nested predicate pushdown, so they are removed. + val newFilters = filters.filter(!_.containsNestedColumn) + val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, newFilters)) conjunctionOptional.map { conjunction => // Then tries to build a single ORC `SearchArgument` for the conjunction predicate. // The input predicate is fully convertible. There should not be any empty result in the @@ -222,48 +224,39 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Since ORC 1.5.0 (ORC-323), we need to quote for column names with `.` characters // in order to distinguish predicate pushdown for nested columns. expression match { - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().equals(quotedName, getType(attribute), castedValue).end()) + case EqualTo(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startAnd().equals(name, getType(name), castedValue).end()) - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end()) + case EqualNullSafe(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startAnd().nullSafeEquals(name, getType(name), castedValue).end()) - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end()) + case LessThan(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startAnd().lessThan(name, getType(name), castedValue).end()) - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + case LessThanOrEqual(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startAnd().lessThanEquals(name, getType(name), castedValue).end()) - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + case GreaterThan(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startNot().lessThanEquals(name, getType(name), castedValue).end()) - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end()) + case GreaterThanOrEqual(name, value) if isSearchableType(dataTypeMap(name)) => + val castedValue = castLiteralValue(value, dataTypeMap(name)) + Some(builder.startNot().lessThan(name, getType(name), castedValue).end()) - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - Some(builder.startAnd().isNull(quotedName, getType(attribute)).end()) + case IsNull(name) if isSearchableType(dataTypeMap(name)) => + Some(builder.startAnd().isNull(name, getType(name)).end()) - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - Some(builder.startNot().isNull(quotedName, getType(attribute)).end()) + case IsNotNull(name) if isSearchableType(dataTypeMap(name)) => + Some(builder.startNot().isNull(name, getType(name)).end()) - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteIfNeeded(attribute) - val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - Some(builder.startAnd().in(quotedName, getType(attribute), + case In(name, values) if isSearchableType(dataTypeMap(name)) => + val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(name))) + Some(builder.startAnd().in(name, getType(name), castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) case _ => None diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala index c32f024476e69..60c5b7a266c51 100644 --- a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala +++ b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources.orc -import java.sql.Date - import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch import org.apache.hadoop.hive.ql.io.sarg.{SearchArgument => OrcSearchArgument} @@ -26,6 +24,7 @@ import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf.{Operator => OrcOperator} import org.apache.hadoop.hive.serde2.io.{DateWritable, HiveDecimalWritable} import org.apache.spark.sql.catalyst.expressions.SpecializedGetters +import org.apache.spark.sql.execution.datasources.DaysWritable import org.apache.spark.sql.types.Decimal /** @@ -38,7 +37,9 @@ private[sql] object OrcShimUtils { private[sql] type Operator = OrcOperator private[sql] type SearchArgument = OrcSearchArgument - def getSqlDate(value: Any): Date = value.asInstanceOf[DateWritable].get + def getGregorianDays(value: Any): Int = { + new DaysWritable(value.asInstanceOf[DateWritable]).gregorianDays + } def getDecimal(value: Any): Decimal = { val decimal = value.asInstanceOf[HiveDecimalWritable].getHiveDecimal() @@ -47,13 +48,13 @@ private[sql] object OrcShimUtils { def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritable = { if (reuseObj) { - val result = new DateWritable() + val result = new DaysWritable() (getter, ordinal) => result.set(getter.getInt(ordinal)) result } else { (getter: SpecializedGetters, ordinal: Int) => - new DateWritable(getter.getInt(ordinal)) + new DaysWritable(getter.getInt(ordinal)) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 6b76927bd4150..5ed0cb0407484 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -46,6 +46,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.security.HiveDelegationTokenProvider +import org.apache.spark.sql.internal.SharedState import org.apache.spark.util.ShutdownHookManager /** @@ -130,6 +131,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { UserGroupInformation.getCurrentUser.addCredentials(credentials) } + SharedState.loadHiveConfFile(sparkConf, conf) SessionState.start(sessionState) // Clean up after we exit @@ -188,8 +190,11 @@ private[hive] object SparkSQLCLIDriver extends Logging { // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) - newHiveConf.foreach { kv => - SparkSQLEnv.sqlContext.setConf(kv._1, kv._2) + // We don't propagate hive.metastore.warehouse.dir, because it might has been adjusted in + // [[SharedState.loadHiveConfFile]] based on the user specified or default values of + // spark.sql.warehouse.dir and hive.metastore.warehouse.dir. + for ((k, v) <- newHiveConf if k != "hive.metastore.warehouse.dir") { + SparkSQLEnv.sqlContext.setConf(k, v) } if (sessionState.execString != null) { diff --git a/sql/hive-thriftserver/src/test/noclasspath/hive-site.xml b/sql/hive-thriftserver/src/test/noclasspath/hive-site.xml new file mode 100644 index 0000000000000..d0bf04d36b00d --- /dev/null +++ b/sql/hive-thriftserver/src/test/noclasspath/hive-site.xml @@ -0,0 +1,30 @@ + + + + + + + hive.in.test + true + Internal marker for test. + + + hive.metastore.warehouse.dir + /tmp/hive_one + + diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 43aafc3c8590c..c393054051fc4 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -27,22 +27,23 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.test.HiveTestJars +import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} /** - * A test suite for the `spark-sql` CLI tool. Note that all test cases share the same temporary - * Hive metastore and warehouse. + * A test suite for the `spark-sql` CLI tool. */ -class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { +class CliSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfterEach with Logging { val warehousePath = Utils.createTempDir() val metastorePath = Utils.createTempDir() val scratchDirPath = Utils.createTempDir() + val sparkWareHouseDir = Utils.createTempDir() override def beforeAll(): Unit = { super.beforeAll() @@ -53,14 +54,20 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { override def afterAll(): Unit = { try { - warehousePath.delete() - metastorePath.delete() - scratchDirPath.delete() + Utils.deleteRecursively(warehousePath) + Utils.deleteRecursively(metastorePath) + Utils.deleteRecursively(scratchDirPath) } finally { super.afterAll() } } + override def afterEach(): Unit = { + // Only running `runCliWithin` in a single test case will share the same temporary + // Hive metastore + Utils.deleteRecursively(metastorePath) + } + /** * Run a CLI operation and expect all the queries and expected answers to be returned. * @@ -75,25 +82,35 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { def runCliWithin( timeout: FiniteDuration, extraArgs: Seq[String] = Seq.empty, - errorResponses: Seq[String] = Seq("Error:"))( + errorResponses: Seq[String] = Seq("Error:"), + maybeWarehouse: Option[File] = Some(warehousePath), + useExternalHiveFile: Boolean = false)( queriesAndExpectedAnswers: (String, String)*): Unit = { val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip // Explicitly adds ENTER for each statement to make sure they are actually entered into the CLI. val queriesString = queries.map(_ + "\n").mkString + val extraHive = if (useExternalHiveFile) { + s"--driver-class-path ${System.getProperty("user.dir")}/src/test/noclasspath" + } else { + "" + } + val warehouseConf = + maybeWarehouse.map(dir => s"--hiveconf ${ConfVars.METASTOREWAREHOUSE}=$dir").getOrElse("") val command = { val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator) val jdbcUrl = s"jdbc:derby:;databaseName=$metastorePath;create=true" s"""$cliScript | --master local | --driver-java-options -Dderby.system.durability=test + | $extraHive | --conf spark.ui.enabled=false | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl - | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath | --hiveconf ${ConfVars.SCRATCHDIR}=$scratchDirPath | --hiveconf conf1=conftest | --hiveconf conf2=1 + | $warehouseConf """.stripMargin.split("\\s+").toSeq ++ extraArgs } @@ -159,6 +176,54 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { } } + test("load warehouse dir from hive-site.xml") { + runCliWithin(1.minute, maybeWarehouse = None, useExternalHiveFile = true)( + "desc database default;" -> "hive_one", + "set spark.sql.warehouse.dir;" -> "hive_one") + } + + test("load warehouse dir from --hiveconf") { + // --hiveconf will overrides hive-site.xml + runCliWithin(2.minute, useExternalHiveFile = true)( + "desc database default;" -> warehousePath.getAbsolutePath, + "create database cliTestDb;" -> "", + "desc database cliTestDb;" -> warehousePath.getAbsolutePath, + "set spark.sql.warehouse.dir;" -> warehousePath.getAbsolutePath) + } + + test("load warehouse dir from --conf spark(.hadoop).hive.*") { + // override conf from hive-site.xml + runCliWithin( + 2.minute, + extraArgs = Seq("--conf", s"spark.hadoop.${ConfVars.METASTOREWAREHOUSE}=$sparkWareHouseDir"), + maybeWarehouse = None, + useExternalHiveFile = true)( + "desc database default;" -> sparkWareHouseDir.getAbsolutePath, + "create database cliTestDb;" -> "", + "desc database cliTestDb;" -> sparkWareHouseDir.getAbsolutePath, + "set spark.sql.warehouse.dir;" -> sparkWareHouseDir.getAbsolutePath) + + // override conf from --hiveconf too + runCliWithin( + 2.minute, + extraArgs = Seq("--conf", s"spark.${ConfVars.METASTOREWAREHOUSE}=$sparkWareHouseDir"))( + "desc database default;" -> sparkWareHouseDir.getAbsolutePath, + "create database cliTestDb;" -> "", + "desc database cliTestDb;" -> sparkWareHouseDir.getAbsolutePath, + "set spark.sql.warehouse.dir;" -> sparkWareHouseDir.getAbsolutePath) + } + + test("load warehouse dir from spark.sql.warehouse.dir") { + // spark.sql.warehouse.dir overrides all hive ones + runCliWithin( + 2.minute, + extraArgs = + Seq("--conf", + s"${StaticSQLConf.WAREHOUSE_PATH.key}=${sparkWareHouseDir}1", + "--conf", s"spark.hadoop.${ConfVars.METASTOREWAREHOUSE}=${sparkWareHouseDir}2"))( + "desc database default;" -> sparkWareHouseDir.getAbsolutePath.concat("1")) + } + test("Simple commands") { val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") @@ -308,19 +373,6 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { ) } - test("SPARK-21451: spark.sql.warehouse.dir should respect options in --hiveconf") { - runCliWithin(1.minute)("set spark.sql.warehouse.dir;" -> warehousePath.getAbsolutePath) - } - - test("SPARK-21451: Apply spark.hadoop.* configurations") { - val tmpDir = Utils.createTempDir(namePrefix = "SPARK-21451") - runCliWithin( - 1.minute, - Seq("--conf", s"spark.hadoop.${ConfVars.METASTOREWAREHOUSE}=$tmpDir"))( - "set spark.sql.warehouse.dir;" -> tmpDir.getAbsolutePath) - tmpDir.delete() - } - test("Support hive.aux.jars.path") { val hiveContribJar = HiveTestJars.getHiveContribJar().getCanonicalPath runCliWithin( diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index d9ac9ab441f0c..15cc3109da3f7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -23,6 +23,7 @@ import java.util.{Locale, MissingFormatArgumentException} import scala.util.control.NonFatal +import org.apache.commons.io.FileUtils import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.spark.SparkException @@ -53,6 +54,15 @@ import org.apache.spark.sql.types._ */ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer { + override protected def testFile(fileName: String): String = { + val url = Thread.currentThread().getContextClassLoader.getResource(fileName) + // Copy to avoid URISyntaxException during accessing the resources in `sql/core` + val file = File.createTempFile("thriftserver-test", ".data") + file.deleteOnExit() + FileUtils.copyURLToFile(url, file) + file.getAbsolutePath + } + /** List of test cases to ignore, in lower cases. */ override def blackList: Set[String] = super.blackList ++ Set( // Missing UDF @@ -80,8 +90,6 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ // We do not test with configSet. withJdbcStatement { statement => - loadTestData(statement) - configSet.foreach { case (k, v) => statement.execute(s"SET $k = $v") } @@ -262,61 +270,6 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ } } - /** Load built-in test tables. */ - private def loadTestData(statement: Statement): Unit = { - // Prepare the data - statement.execute( - """ - |CREATE OR REPLACE TEMPORARY VIEW testdata as - |SELECT id AS key, CAST(id AS string) AS value FROM range(1, 101) - """.stripMargin) - statement.execute( - """ - |CREATE OR REPLACE TEMPORARY VIEW arraydata as - |SELECT * FROM VALUES - |(ARRAY(1, 2, 3), ARRAY(ARRAY(1, 2, 3))), - |(ARRAY(2, 3, 4), ARRAY(ARRAY(2, 3, 4))) AS v(arraycol, nestedarraycol) - """.stripMargin) - statement.execute( - """ - |CREATE OR REPLACE TEMPORARY VIEW mapdata as - |SELECT * FROM VALUES - |MAP(1, 'a1', 2, 'b1', 3, 'c1', 4, 'd1', 5, 'e1'), - |MAP(1, 'a2', 2, 'b2', 3, 'c2', 4, 'd2'), - |MAP(1, 'a3', 2, 'b3', 3, 'c3'), - |MAP(1, 'a4', 2, 'b4'), - |MAP(1, 'a5') AS v(mapcol) - """.stripMargin) - statement.execute( - s""" - |CREATE TEMPORARY VIEW aggtest - | (a int, b float) - |USING csv - |OPTIONS (path '${baseResourcePath.getParent}/test-data/postgresql/agg.data', - | header 'false', delimiter '\t') - """.stripMargin) - statement.execute( - s""" - |CREATE OR REPLACE TEMPORARY VIEW onek - | (unique1 int, unique2 int, two int, four int, ten int, twenty int, hundred int, - | thousand int, twothousand int, fivethous int, tenthous int, odd int, even int, - | stringu1 string, stringu2 string, string4 string) - |USING csv - |OPTIONS (path '${baseResourcePath.getParent}/test-data/postgresql/onek.data', - | header 'false', delimiter '\t') - """.stripMargin) - statement.execute( - s""" - |CREATE OR REPLACE TEMPORARY VIEW tenk1 - | (unique1 int, unique2 int, two int, four int, ten int, twenty int, hundred int, - | thousand int, twothousand int, fivethous int, tenthous int, odd int, even int, - | stringu1 string, stringu2 string, string4 string) - |USING csv - | OPTIONS (path '${baseResourcePath.getParent}/test-data/postgresql/tenk.data', - | header 'false', delimiter '\t') - """.stripMargin) - } - // Returns true if sql is retrieving data. private def isNeedSort(sql: String): Boolean = { val upperCase = sql.toUpperCase(Locale.ROOT) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala new file mode 100644 index 0000000000000..02a5117f005e8 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.SparkContext +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{SparkSession, SQLContext} + + +/** + * An instance of the Spark SQL execution engine that integrates with data stored in Hive. + * Configuration for Hive is read from hive-site.xml on the classpath. + */ +@deprecated("Use SparkSession.builder.enableHiveSupport instead", "2.0.0") +class HiveContext private[hive](_sparkSession: SparkSession) + extends SQLContext(_sparkSession) with Logging { + + self => + + def this(sc: SparkContext) = { + this(SparkSession.builder().sparkContext(HiveUtils.withHiveExternalCatalog(sc)).getOrCreate()) + } + + def this(sc: JavaSparkContext) = this(sc.sc) + + /** + * Returns a new HiveContext as new session, which will have separated SQLConf, UDF/UDAF, + * temporary tables and SessionState, but sharing the same CacheManager, IsolatedClientLoader + * and Hive client (both of execution and metadata) with existing HiveContext. + */ + override def newSession(): HiveContext = { + new HiveContext(sparkSession.newSession()) + } + + /** + * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, + * Spark SQL or the external data source library it uses might cache certain metadata about a + * table, such as the location of blocks. When those change outside of Spark SQL, users should + * call this function to invalidate the cache. + * + * @since 1.3.0 + */ + def refreshTable(tableName: String): Unit = { + sparkSession.catalog.refreshTable(tableName) + } + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index be6d824ece684..2faf42028f3a2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -861,6 +861,11 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat client.listTables(db, pattern) } + override def listViews(db: String, pattern: String): Seq[String] = withClient { + requireDbExists(db) + client.listTablesByType(db, pattern, CatalogTableType.VIEW) + } + override def loadTable( db: String, table: String, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index e217c5279046e..16e9014340244 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -18,8 +18,6 @@ package org.apache.spark.sql.hive import java.lang.reflect.{ParameterizedType, Type, WildcardType} -import java.time.LocalDate -import java.util.Calendar import scala.collection.JavaConverters._ @@ -34,6 +32,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.datasources.DaysWritable import org.apache.spark.sql.types import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -182,33 +181,6 @@ import org.apache.spark.unsafe.types.UTF8String */ private[hive] trait HiveInspectors { - private final val JULIAN_CUTOVER_DAY = - rebaseGregorianToJulianDays(DateTimeUtils.GREGORIAN_CUTOVER_DAY.toInt) - - private def rebaseJulianToGregorianDays(daysSinceEpoch: Int): Int = { - val localDate = LocalDate.ofEpochDay(daysSinceEpoch) - val utcCal = new Calendar.Builder() - .setCalendarType("gregory") - .setTimeZone(DateTimeUtils.TimeZoneUTC) - .setDate(localDate.getYear, localDate.getMonthValue - 1, localDate.getDayOfMonth) - .build() - Math.toIntExact(Math.floorDiv(utcCal.getTimeInMillis, DateTimeConstants.MILLIS_PER_DAY)) - } - - private def rebaseGregorianToJulianDays(daysSinceEpoch: Int): Int = { - val millis = Math.multiplyExact(daysSinceEpoch, DateTimeConstants.MILLIS_PER_DAY) - val utcCal = new Calendar.Builder() - .setCalendarType("gregory") - .setTimeZone(DateTimeUtils.TimeZoneUTC) - .setInstant(millis) - .build() - val localDate = LocalDate.of( - utcCal.get(Calendar.YEAR), - utcCal.get(Calendar.MONTH) + 1, - utcCal.get(Calendar.DAY_OF_MONTH)) - Math.toIntExact(localDate.toEpochDay) - } - def javaTypeToDataType(clz: Type): DataType = clz match { // writable case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType @@ -646,14 +618,7 @@ private[hive] trait HiveInspectors { case x: DateObjectInspector if x.preferWritable() => data: Any => { if (data != null) { - // Rebasing written days via conversion to local dates. - // See the comment for `getDateWritable()`. - val daysSinceEpoch = x.getPrimitiveWritableObject(data).getDays - if (daysSinceEpoch < JULIAN_CUTOVER_DAY) { - rebaseJulianToGregorianDays(daysSinceEpoch) - } else { - daysSinceEpoch - } + new DaysWritable(x.getPrimitiveWritableObject(data)).gregorianDays } else { null } @@ -1045,27 +1010,11 @@ private[hive] trait HiveInspectors { new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) } - private def getDateWritable(value: Any): hiveIo.DateWritable = + private def getDateWritable(value: Any): DaysWritable = if (value == null) { null } else { - // Rebasing days since the epoch to store the same number of days - // as by Spark 2.4 and earlier versions. Spark 3.0 switched to - // Proleptic Gregorian calendar (see SPARK-26651), and as a consequence of that, - // this affects dates before 1582-10-15. Spark 2.4 and earlier versions use - // Julian calendar for dates before 1582-10-15. So, the same local date may - // be mapped to different number of days since the epoch in different calendars. - // For example: - // Proleptic Gregorian calendar: 1582-01-01 -> -141714 - // Julian calendar: 1582-01-01 -> -141704 - // The code below converts -141714 to -141704. - val daysSinceEpoch = value.asInstanceOf[Int] - val rebasedDays = if (daysSinceEpoch < DateTimeUtils.GREGORIAN_CUTOVER_DAY) { - rebaseGregorianToJulianDays(daysSinceEpoch) - } else { - daysSinceEpoch - } - new hiveIo.DateWritable(rebasedDays) + new DaysWritable(value.asInstanceOf[Int]) } private def getTimestampWritable(value: Any): hiveIo.TimestampWritable = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index 3beef6b1df457..04a6a8f8aa9a5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -118,9 +118,12 @@ private[hive] object HiveShim { * * @param functionClassName UDF class name * @param instance optional UDF instance which contains additional information (for macro) + * @param clazz optional class instance to create UDF instance */ - private[hive] case class HiveFunctionWrapper(var functionClassName: String, - private var instance: AnyRef = null) extends java.io.Externalizable { + private[hive] case class HiveFunctionWrapper( + var functionClassName: String, + private var instance: AnyRef = null, + private var clazz: Class[_ <: AnyRef] = null) extends java.io.Externalizable { // for Serialization def this() = this(null) @@ -232,8 +235,10 @@ private[hive] object HiveShim { in.readFully(functionInBytes) // deserialize the function object via Hive Utilities + clazz = Utils.getContextOrSparkClassLoader.loadClass(functionClassName) + .asInstanceOf[Class[_ <: AnyRef]] instance = deserializePlan[AnyRef](new java.io.ByteArrayInputStream(functionInBytes), - Utils.getContextOrSparkClassLoader.loadClass(functionClassName)) + clazz) } } @@ -241,8 +246,11 @@ private[hive] object HiveShim { if (instance != null) { instance.asInstanceOf[UDFType] } else { - val func = Utils.getContextOrSparkClassLoader - .loadClass(functionClassName).getConstructor().newInstance().asInstanceOf[UDFType] + if (clazz == null) { + clazz = Utils.getContextOrSparkClassLoader.loadClass(functionClassName) + .asInstanceOf[Class[_ <: AnyRef]] + } + val func = clazz.getConstructor().newInstance().asInstanceOf[UDFType] if (!func.isInstanceOf[UDF]) { // We cache the function if it's no the Simple UDF, // as we always have to create new instance for Simple UDF diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 9c4b8a5819a33..3c20e68278e3e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -65,6 +65,7 @@ private[spark] object HiveUtils extends Logging { .doc("Version of the Hive metastore. Available options are " + "0.12.0 through 2.3.6 and " + "3.0.0 through 3.1.2.") + .version("1.4.0") .stringConf .createWithDefault(builtinHiveVersion) @@ -73,6 +74,7 @@ private[spark] object HiveUtils extends Logging { // already rely on this config. val FAKE_HIVE_VERSION = buildConf("spark.sql.hive.version") .doc(s"deprecated, please use ${HIVE_METASTORE_VERSION.key} to get the Hive version in Spark.") + .version("1.1.1") .stringConf .createWithDefault(builtinHiveVersion) @@ -89,12 +91,14 @@ private[spark] object HiveUtils extends Logging { | Use Hive jars of specified version downloaded from Maven repositories. | 3. A classpath in the standard format for both Hive and Hadoop. """.stripMargin) + .version("1.4.0") .stringConf .createWithDefault("builtin") val CONVERT_METASTORE_PARQUET = buildConf("spark.sql.hive.convertMetastoreParquet") .doc("When set to true, the built-in Parquet reader and writer are used to process " + "parquet tables created by using the HiveQL syntax, instead of Hive serde.") + .version("1.1.1") .booleanConf .createWithDefault(true) @@ -103,12 +107,14 @@ private[spark] object HiveUtils extends Logging { .doc("When true, also tries to merge possibly different but compatible Parquet schemas in " + "different Parquet data files. This configuration is only effective " + "when \"spark.sql.hive.convertMetastoreParquet\" is true.") + .version("1.3.1") .booleanConf .createWithDefault(false) val CONVERT_METASTORE_ORC = buildConf("spark.sql.hive.convertMetastoreOrc") .doc("When set to true, the built-in ORC reader and writer are used to process " + "ORC tables created by using the HiveQL syntax, instead of Hive serde.") + .version("2.0.0") .booleanConf .createWithDefault(true) @@ -118,6 +124,7 @@ private[spark] object HiveUtils extends Logging { "`spark.sql.hive.convertMetastoreOrc` is true, the built-in ORC/Parquet writer is used" + "to process inserting into partitioned ORC/Parquet tables created by using the HiveSQL " + "syntax.") + .version("3.0.0") .booleanConf .createWithDefault(true) @@ -126,6 +133,7 @@ private[spark] object HiveUtils extends Logging { "instead of Hive serde in CTAS. This flag is effective only if " + "`spark.sql.hive.convertMetastoreParquet` or `spark.sql.hive.convertMetastoreOrc` is " + "enabled respectively for Parquet and ORC formats") + .version("3.0.0") .booleanConf .createWithDefault(true) @@ -135,6 +143,7 @@ private[spark] object HiveUtils extends Logging { "that should be shared is JDBC drivers that are needed to talk to the metastore. Other " + "classes that need to be shared are those that interact with classes that are already " + "shared. For example, custom appenders that are used by log4j.") + .version("1.4.0") .stringConf .toSequence .createWithDefault(jdbcPrefixes) @@ -146,12 +155,14 @@ private[spark] object HiveUtils extends Logging { .doc("A comma separated list of class prefixes that should explicitly be reloaded for each " + "version of Hive that Spark SQL is communicating with. For example, Hive UDFs that are " + "declared in a prefix that typically would be shared (i.e. org.apache.spark.*).") + .version("1.4.0") .stringConf .toSequence .createWithDefault(Nil) val HIVE_THRIFT_SERVER_ASYNC = buildConf("spark.sql.hive.thriftServer.async") .doc("When set to true, Hive Thrift server executes SQL queries in an asynchronous way.") + .version("1.5.0") .booleanConf .createWithDefault(true) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index e31dffa4795c5..3ea80eaf6f714 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -61,6 +61,15 @@ private[hive] trait HiveClient { /** Returns the names of tables in the given database that matches the given pattern. */ def listTables(dbName: String, pattern: String): Seq[String] + /** + * Returns the names of tables with specific tableType in the given database that matches + * the given pattern. + */ + def listTablesByType( + dbName: String, + pattern: String, + tableType: CatalogTableType): Seq[String] + /** Sets the name of current database. */ def setCurrentDatabase(databaseName: String): Unit 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 4a3e813740b48..6ad5e9d3c9080 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 @@ -755,6 +755,22 @@ private[hive] class HiveClientImpl( client.getTablesByPattern(dbName, pattern).asScala } + override def listTablesByType( + dbName: String, + pattern: String, + tableType: CatalogTableType): Seq[String] = withHiveState { + try { + // Try with Hive API getTablesByType first, it's supported from Hive 2.3+. + shim.getTablesByType(client, dbName, pattern, toHiveTableType(tableType)) + } catch { + case _: UnsupportedOperationException => + // Fallback to filter logic if getTablesByType not supported. + val tableNames = client.getTablesByPattern(dbName, pattern).asScala + val tables = getTablesByName(dbName, tableNames).filter(_.tableType == tableType) + tables.map(_.identifier.table) + } + } + /** * Runs the specified SQL query using Hive. */ @@ -1011,25 +1027,29 @@ private[hive] object HiveClientImpl extends Logging { private def toOutputFormat(name: String) = Utils.classForName[org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]](name) + def toHiveTableType(catalogTableType: CatalogTableType): HiveTableType = { + catalogTableType match { + case CatalogTableType.EXTERNAL => HiveTableType.EXTERNAL_TABLE + case CatalogTableType.MANAGED => HiveTableType.MANAGED_TABLE + case CatalogTableType.VIEW => HiveTableType.VIRTUAL_VIEW + case t => + throw new IllegalArgumentException( + s"Unknown table type is found at toHiveTableType: $t") + } + } + /** * Converts the native table metadata representation format CatalogTable to Hive's Table. */ def toHiveTable(table: CatalogTable, userName: Option[String] = None): HiveTable = { val hiveTable = new HiveTable(table.database, table.identifier.table) + hiveTable.setTableType(toHiveTableType(table.tableType)) // For EXTERNAL_TABLE, we also need to set EXTERNAL field in the table properties. // Otherwise, Hive metastore will change the table to a MANAGED_TABLE. // (metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1095-L1105) - hiveTable.setTableType(table.tableType match { - case CatalogTableType.EXTERNAL => - hiveTable.setProperty("EXTERNAL", "TRUE") - HiveTableType.EXTERNAL_TABLE - case CatalogTableType.MANAGED => - HiveTableType.MANAGED_TABLE - case CatalogTableType.VIEW => HiveTableType.VIRTUAL_VIEW - case t => - throw new IllegalArgumentException( - s"Unknown table type is found at toHiveTable: $t") - }) + if (table.tableType == CatalogTableType.EXTERNAL) { + hiveTable.setProperty("EXTERNAL", "TRUE") + } // Note: In Hive the schema and partition columns must be disjoint sets val (partCols, schema) = table.schema.map(toHiveColumn).partition { c => table.partitionColumnNames.contains(c.getName) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 50ce536a160c8..2b806609426a1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -29,6 +29,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.IMetaStoreClient +import org.apache.hadoop.hive.metastore.TableType import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, MetaException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.io.AcidUtils @@ -90,6 +91,12 @@ private[client] sealed abstract class Shim { def alterPartitions(hive: Hive, tableName: String, newParts: JList[Partition]): Unit + def getTablesByType( + hive: Hive, + dbName: String, + pattern: String, + tableType: TableType): Seq[String] + def createPartitions( hive: Hive, db: String, @@ -363,6 +370,15 @@ private[client] class Shim_v0_12 extends Shim with Logging { conf.getIntVar(HiveConf.ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY) * 1000L } + override def getTablesByType( + hive: Hive, + dbName: String, + pattern: String, + tableType: TableType): Seq[String] = { + throw new UnsupportedOperationException("Hive 2.2 and lower versions don't support " + + "getTablesByType. Please use Hive 2.3 or higher version.") + } + override def loadPartition( hive: Hive, loadPath: Path, @@ -1220,7 +1236,24 @@ private[client] class Shim_v2_1 extends Shim_v2_0 { private[client] class Shim_v2_2 extends Shim_v2_1 -private[client] class Shim_v2_3 extends Shim_v2_1 +private[client] class Shim_v2_3 extends Shim_v2_1 { + private lazy val getTablesByTypeMethod = + findMethod( + classOf[Hive], + "getTablesByType", + classOf[String], + classOf[String], + classOf[TableType]) + + override def getTablesByType( + hive: Hive, + dbName: String, + pattern: String, + tableType: TableType): Seq[String] = { + getTablesByTypeMethod.invoke(hive, dbName, pattern, tableType) + .asInstanceOf[JList[String]].asScala + } +} private[client] class Shim_v3_0 extends Shim_v2_3 { // Spark supports only non-ACID operations 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 801be64702519..19f439598142e 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 @@ -106,7 +106,7 @@ case class InsertIntoHiveTable( } // un-cache this table. - sparkSession.catalog.uncacheTable(table.identifier.quotedString) + CommandUtils.uncacheTableOrView(sparkSession, table.identifier.quotedString) sparkSession.sessionState.catalog.refreshTable(table.identifier) CommandUtils.updateTableStats(sparkSession, table) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala index cd1bffb6b7ab7..f9c514567c639 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory.newBuilder import org.apache.spark.SparkException import org.apache.spark.internal.Logging +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.quoteIfNeeded import org.apache.spark.sql.execution.datasources.orc.{OrcFilters => DatasourceOrcFilters} import org.apache.spark.sql.execution.datasources.orc.OrcFilters.buildTree import org.apache.spark.sql.hive.HiveUtils @@ -73,9 +74,11 @@ private[orc] object OrcFilters extends Logging { if (HiveUtils.isHive23) { DatasourceOrcFilters.createFilter(schema, filters).asInstanceOf[Option[SearchArgument]] } else { - val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + val dataTypeMap = schema.map(f => quoteIfNeeded(f.name) -> f.dataType).toMap + // TODO (SPARK-25557): ORC doesn't support nested predicate pushdown, so they are removed. + val newFilters = filters.filter(!_.containsNestedColumn) // Combines all convertible filters using `And` to produce a single conjunction - val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) + val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, newFilters)) conjunctionOptional.map { conjunction => // Then tries to build a single ORC `SearchArgument` for the conjunction predicate. // The input predicate is fully convertible. There should not be any empty result in the diff --git a/sql/hive/src/test/noclasspath/TestUDTF-spark-26560.jar b/sql/hive/src/test/noclasspath/TestUDTF-spark-26560.jar deleted file mode 100644 index b73b17d5c7880..0000000000000 Binary files a/sql/hive/src/test/noclasspath/TestUDTF-spark-26560.jar and /dev/null differ diff --git a/sql/hive/src/test/noclasspath/hive-test-udfs.jar b/sql/hive/src/test/noclasspath/hive-test-udfs.jar new file mode 100644 index 0000000000000..a5bfa456f6686 Binary files /dev/null and b/sql/hive/src/test/noclasspath/hive-test-udfs.jar differ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala new file mode 100644 index 0000000000000..a80db765846e9 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala @@ -0,0 +1,103 @@ +/* + * 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 org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} + + +class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEach { + + override protected val enableAutoThreadAudit = false + private var sc: SparkContext = null + private var hc: HiveContext = null + + override def beforeAll(): Unit = { + super.beforeAll() + sc = SparkContext.getOrCreate(new SparkConf().setMaster("local").setAppName("test")) + HiveUtils.newTemporaryConfiguration(useInMemoryDerby = true).foreach { case (k, v) => + sc.hadoopConfiguration.set(k, v) + } + hc = new HiveContext(sc) + } + + override def afterEach(): Unit = { + try { + hc.sharedState.cacheManager.clearCache() + hc.sessionState.catalog.reset() + } finally { + super.afterEach() + } + } + + override def afterAll(): Unit = { + try { + sc = null + hc = null + } finally { + super.afterAll() + } + } + + test("basic operations") { + val _hc = hc + import _hc.implicits._ + val df1 = (1 to 20).map { i => (i, i) }.toDF("a", "x") + val df2 = (1 to 100).map { i => (i, i % 10, i % 2 == 0) }.toDF("a", "b", "c") + .select($"a", $"b") + .filter($"a" > 10 && $"b" > 6 && $"c") + val df3 = df1.join(df2, "a") + val res = df3.collect() + val expected = Seq((18, 18, 8)).toDF("a", "x", "b").collect() + assert(res.toSeq == expected.toSeq) + df3.createOrReplaceTempView("mai_table") + val df4 = hc.table("mai_table") + val res2 = df4.collect() + assert(res2.toSeq == expected.toSeq) + } + + test("basic DDLs") { + val _hc = hc + import _hc.implicits._ + val databases = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) + assert(databases.toSeq == Seq("default")) + hc.sql("CREATE DATABASE mee_db") + hc.sql("USE mee_db") + val databases2 = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) + assert(databases2.toSet == Set("default", "mee_db")) + val df = (1 to 10).map { i => ("bob" + i.toString, i) }.toDF("name", "age") + df.createOrReplaceTempView("mee_table") + hc.sql("CREATE TABLE moo_table (name string, age int)") + hc.sql("INSERT INTO moo_table SELECT * FROM mee_table") + assert( + hc.sql("SELECT * FROM moo_table order by name").collect().toSeq == + df.collect().toSeq.sortBy(_.getString(0))) + val tables = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0)) + assert(tables.toSet == Set("moo_table", "mee_table")) + hc.sql("DROP TABLE moo_table") + hc.sql("DROP TABLE mee_table") + val tables2 = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0)) + assert(tables2.isEmpty) + hc.sql("USE default") + hc.sql("DROP DATABASE mee_db CASCADE") + val databases3 = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) + assert(databases3.toSeq == Seq("default")) + } + +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala index 94a55b911f092..743cdbd6457d7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.adaptive.AdaptiveTestUtils.assertExceptionMessage import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -99,7 +100,7 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi val e = intercept[SparkException] { sql("select * from test").count() } - assert(e.getMessage.contains("FileNotFoundException")) + assertExceptionMessage(e, "FileNotFoundException") // Test refreshing the cache. spark.catalog.refreshTable("test") @@ -114,7 +115,7 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi val e2 = intercept[SparkException] { sql("select * from test").count() } - assert(e2.getMessage.contains("FileNotFoundException")) + assertExceptionMessage(e2, "FileNotFoundException") spark.catalog.refreshByPath(dir.getAbsolutePath) assert(sql("select * from test").count() == 3) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 9cd56f1745afc..95e99c653d6f6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -97,7 +97,7 @@ class HiveMetastoreCatalogSuite extends TestHiveSingleton with SQLTestUtils { |c22 map, |c23 struct, |c24 struct - |) + |) USING hive """.stripMargin) val schema = hiveClient.getTable("default", "t").schema diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSharedStateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSharedStateSuite.scala index 6e2dcfc04d498..78535b094b83d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSharedStateSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSharedStateSuite.scala @@ -45,7 +45,6 @@ class HiveSharedStateSuite extends SparkFunSuite { GLOBAL_TEMP_DATABASE.key -> tmpDb) val state = new SharedState(sc, initialConfigs) - assert(state.warehousePath !== invalidPath, "warehouse path can't determine by session options") assert(sc.conf.get(WAREHOUSE_PATH.key) !== invalidPath, "warehouse conf in session options can't affect application wide spark conf") assert(sc.hadoopConfiguration.get(ConfVars.METASTOREWAREHOUSE.varname) !== invalidPath, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala index 50c9018e4e2a9..1e31e8b1bf234 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala @@ -42,16 +42,43 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet } test("view") { - withView("v1") { - sql("CREATE VIEW v1 AS SELECT 1 AS a") - checkCreateHiveTableOrView("v1", "VIEW") + Seq(true, false).foreach { serde => + withView("v1") { + sql("CREATE VIEW v1 AS SELECT 1 AS a") + checkCreateView("v1", serde) + } } } - test("view with output columns") { - withView("v1") { - sql("CREATE VIEW v1 (b) AS SELECT 1 AS a") - checkCreateHiveTableOrView("v1", "VIEW") + test("view with output columns") { + Seq(true, false).foreach { serde => + withView("v1") { + sql("CREATE VIEW v1 (a, b COMMENT 'b column') AS SELECT 1 AS a, 2 AS b") + checkCreateView("v1", serde) + } + } + } + + test("view with table comment and properties") { + Seq(true, false).foreach { serde => + withView("v1") { + sql( + s""" + |CREATE VIEW v1 ( + | c1 COMMENT 'bla', + | c2 + |) + |COMMENT 'table comment' + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + |AS SELECT 1 AS c1, '2' AS c2 + """.stripMargin + ) + + checkCreateView("v1", serde) + } } } @@ -69,7 +96,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateHiveTableOrView("t1") + checkCreateTable("t1", serde = true) } } @@ -89,7 +116,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateHiveTableOrView("t1") + checkCreateTable("t1", serde = true) } } } @@ -109,7 +136,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateHiveTableOrView("t1") + checkCreateTable("t1", serde = true) } } @@ -127,7 +154,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateHiveTableOrView("t1") + checkCreateTable("t1", serde = true) } } @@ -142,7 +169,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateHiveTableOrView("t1") + checkCreateTable("t1", serde = true) } } @@ -164,7 +191,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateHiveTableOrView("t1") + checkCreateTable("t1", serde = true) } } @@ -177,7 +204,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet |INTO 2 BUCKETS """.stripMargin ) - checkCreateHiveTableOrView("t1") + checkCreateTable("t1", serde = true) } } @@ -222,27 +249,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet val shownDDL = getShowDDL("SHOW CREATE TABLE t1") assert(shownDDL == "CREATE TABLE `default`.`t1` (`a` STRUCT<`b`: STRING>)") - checkCreateHiveTableOrView("t1") - } - } - - /** - * This method compares the given table with the table created by the DDL generated by - * `SHOW CREATE TABLE AS SERDE`. - */ - private def checkCreateHiveTableOrView(tableName: String, checkType: String = "TABLE"): Unit = { - val table = TableIdentifier(tableName, Some("default")) - val db = table.database.getOrElse("default") - val expected = spark.sharedState.externalCatalog.getTable(db, table.table) - val shownDDL = sql(s"SHOW CREATE TABLE ${table.quotedString} AS SERDE").head().getString(0) - sql(s"DROP $checkType ${table.quotedString}") - - try { - sql(shownDDL) - val actual = spark.sharedState.externalCatalog.getTable(db, table.table) - checkCatalogTables(expected, actual) - } finally { - sql(s"DROP $checkType IF EXISTS ${table.table}") + checkCreateTable("t1", serde = true) } } @@ -344,7 +351,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet ) val cause = intercept[AnalysisException] { - checkCreateHiveTableOrView("t1") + checkCreateTable("t1", serde = true) } assert(cause.getMessage.contains("Use `SHOW CREATE TABLE` without `AS SERDE` instead")) @@ -446,27 +453,6 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet } } - test("hive view is not supported by show create table without as serde") { - withTable("t1") { - withView("v1") { - sql("CREATE TABLE t1 (c1 STRING, c2 STRING)") - - createRawHiveTable( - s""" - |CREATE VIEW v1 - |AS SELECT * from t1 - """.stripMargin - ) - - val cause = intercept[AnalysisException] { - sql("SHOW CREATE TABLE v1") - } - - assert(cause.getMessage.contains("view isn't supported")) - } - } - } - test("partitioned, bucketed hive table in Spark DDL") { withTable("t1") { sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 31ff62ed0a530..8b97489e2d818 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -787,7 +787,7 @@ object SPARK_18360 { .enableHiveSupport().getOrCreate() val defaultDbLocation = spark.catalog.getDatabase("default").locationUri - assert(new Path(defaultDbLocation) == new Path(spark.sharedState.warehousePath)) + assert(new Path(defaultDbLocation) == new Path(spark.conf.get(WAREHOUSE_PATH))) val hiveClient = spark.sharedState.externalCatalog.unwrapped.asInstanceOf[HiveExternalCatalog].client diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala new file mode 100644 index 0000000000000..ee8e6f4f78be5 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala @@ -0,0 +1,190 @@ +/* + * 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 org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.{IntegerType, StringType} +import org.apache.spark.util.Utils + +class HiveUDFDynamicLoadSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + case class UDFTestInformation( + identifier: String, + funcName: String, + className: String, + fnVerifyQuery: () => Unit, + fnCreateHiveUDFExpression: () => Expression) + + private val udfTestInfos: Seq[UDFTestInformation] = Array( + // UDF + // UDFExampleAdd2 is slightly modified version of UDFExampleAdd in hive/contrib, + // which adds two integers or doubles. + UDFTestInformation( + "UDF", + "udf_add2", + "org.apache.hadoop.hive.contrib.udf.example.UDFExampleAdd2", + () => { + checkAnswer(sql("SELECT udf_add2(1, 2)"), Row(3) :: Nil) + }, + () => { + HiveSimpleUDF( + "default.udf_add2", + HiveFunctionWrapper("org.apache.hadoop.hive.contrib.udf.example.UDFExampleAdd2"), + Array( + AttributeReference("a", IntegerType, nullable = false)(), + AttributeReference("b", IntegerType, nullable = false)())) + }), + + // GenericUDF + // GenericUDFTrim2 is cloned version of GenericUDFTrim in hive/contrib. + UDFTestInformation( + "GENERIC_UDF", + "generic_udf_trim2", + "org.apache.hadoop.hive.contrib.udf.example.GenericUDFTrim2", + () => { + checkAnswer(sql("SELECT generic_udf_trim2(' hello ')"), Row("hello") :: Nil) + }, + () => { + HiveGenericUDF( + "default.generic_udf_trim2", + HiveFunctionWrapper("org.apache.hadoop.hive.contrib.udf.example.GenericUDFTrim2"), + Array(AttributeReference("a", StringType, nullable = false)()) + ) + } + ), + + // AbstractGenericUDAFResolver + // GenericUDAFSum2 is cloned version of GenericUDAFSum in hive/exec. + UDFTestInformation( + "GENERIC_UDAF", + "generic_udaf_sum2", + "org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum2", + () => { + import spark.implicits._ + val df = Seq((0: Integer) -> 0, (1: Integer) -> 1, (2: Integer) -> 2, (3: Integer) -> 3) + .toDF("key", "value").createOrReplaceTempView("t") + checkAnswer(sql("SELECT generic_udaf_sum2(value) FROM t GROUP BY key % 2"), + Row(2) :: Row(4) :: Nil) + }, + () => { + HiveUDAFFunction( + "default.generic_udaf_sum2", + HiveFunctionWrapper("org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum2"), + Array(AttributeReference("a", IntegerType, nullable = false)()) + ) + } + ), + + // UDAF + // UDAFExampleMax2 is cloned version of UDAFExampleMax in hive/contrib. + UDFTestInformation( + "UDAF", + "udaf_max2", + "org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax2", + () => { + import spark.implicits._ + val df = Seq((0: Integer) -> 0, (1: Integer) -> 1, (2: Integer) -> 2, (3: Integer) -> 3) + .toDF("key", "value").createOrReplaceTempView("t") + checkAnswer(sql("SELECT udaf_max2(value) FROM t GROUP BY key % 2"), + Row(2) :: Row(3) :: Nil) + }, + () => { + HiveUDAFFunction( + "default.udaf_max2", + HiveFunctionWrapper("org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax2"), + Array(AttributeReference("a", IntegerType, nullable = false)()), + isUDAFBridgeRequired = true + ) + } + ), + + // GenericUDTF + // GenericUDTFCount3 is slightly modified version of GenericUDTFCount2 in hive/contrib, + // which emits the count for three times. + UDFTestInformation( + "GENERIC_UDTF", + "udtf_count3", + "org.apache.hadoop.hive.contrib.udtf.example.GenericUDTFCount3", + () => { + checkAnswer( + sql("SELECT udtf_count3(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Row(3) :: Nil) + }, + () => { + HiveGenericUDTF( + "default.udtf_count3", + HiveFunctionWrapper("org.apache.hadoop.hive.contrib.udtf.example.GenericUDTFCount3"), + Array.empty[Expression] + ) + } + ) + ) + + udfTestInfos.foreach { udfInfo => + // The test jars are built from below commit: + // https://github.com/HeartSaVioR/hive/commit/12f3f036b6efd0299cd1d457c0c0a65e0fd7e5f2 + // which contain new UDF classes to be dynamically loaded and tested via Spark. + + // This jar file should not be placed to the classpath. + val jarPath = "src/test/noclasspath/hive-test-udfs.jar" + val jarUrl = s"file://${System.getProperty("user.dir")}/$jarPath" + + test("Spark should be able to run Hive UDF using jar regardless of " + + s"current thread context classloader (${udfInfo.identifier}") { + Utils.withContextClassLoader(Utils.getSparkClassLoader) { + withUserDefinedFunction(udfInfo.funcName -> false) { + val sparkClassLoader = Thread.currentThread().getContextClassLoader + + sql(s"CREATE FUNCTION ${udfInfo.funcName} AS '${udfInfo.className}' USING JAR '$jarUrl'") + + assert(Thread.currentThread().getContextClassLoader eq sparkClassLoader) + + // JAR will be loaded at first usage, and it will change the current thread's + // context classloader to jar classloader in sharedState. + // See SessionState.addJar for details. + udfInfo.fnVerifyQuery() + + assert(Thread.currentThread().getContextClassLoader ne sparkClassLoader) + assert(Thread.currentThread().getContextClassLoader eq + spark.sqlContext.sharedState.jarClassLoader) + + val udfExpr = udfInfo.fnCreateHiveUDFExpression() + // force initializing - this is what we do in HiveSessionCatalog + udfExpr.dataType + + // Roll back to the original classloader and run query again. Without this line, the test + // would pass, as thread's context classloader is changed to jar classloader. But thread + // context classloader can be changed from others as well which would fail the query; one + // example is spark-shell, which thread context classloader rolls back automatically. This + // mimics the behavior of spark-shell. + Thread.currentThread().setContextClassLoader(sparkClassLoader) + + udfInfo.fnVerifyQuery() + + val newExpr = udfExpr.makeCopy(udfExpr.productIterator.map(_.asInstanceOf[AnyRef]) + .toArray) + newExpr.dataType + } + } + } + } +} 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 488175a22bad7..c1eab63ec073f 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 @@ -1520,10 +1520,12 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto val ext_tbl = "SPARK_30269_external" withTempDir { dir => withTable(tbl, ext_tbl) { - sql(s"CREATE TABLE $tbl (key INT, value STRING, ds STRING) PARTITIONED BY (ds)") + sql(s"CREATE TABLE $tbl (key INT, value STRING, ds STRING)" + + "USING parquet PARTITIONED BY (ds)") sql( s""" | CREATE TABLE $ext_tbl (key INT, value STRING, ds STRING) + | USING PARQUET | PARTITIONED BY (ds) | LOCATION '${dir.toURI}' """.stripMargin) 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 7471142d4e6fa..d1dd13623650d 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 @@ -154,10 +154,11 @@ class VersionsSuite extends SparkFunSuite with Logging { .client.version.fullVersion.startsWith(version)) } - def table(database: String, tableName: String): CatalogTable = { + def table(database: String, tableName: String, + tableType: CatalogTableType = CatalogTableType.MANAGED): CatalogTable = { CatalogTable( identifier = TableIdentifier(tableName, Some(database)), - tableType = CatalogTableType.MANAGED, + tableType = tableType, schema = new StructType().add("key", "int"), storage = CatalogStorageFormat( locationUri = None, @@ -273,7 +274,9 @@ class VersionsSuite extends SparkFunSuite with Logging { test(s"$version: createTable") { client.createTable(table("default", tableName = "src"), ignoreIfExists = false) - client.createTable(table("default", "temporary"), ignoreIfExists = false) + client.createTable(table("default", tableName = "temporary"), ignoreIfExists = false) + client.createTable(table("default", tableName = "view1", tableType = CatalogTableType.VIEW), + ignoreIfExists = false) } test(s"$version: loadTable") { @@ -389,7 +392,7 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: listTables(database)") { - assert(client.listTables("default") === Seq("src", "temporary")) + assert(client.listTables("default") === Seq("src", "temporary", "view1")) } test(s"$version: listTables(database, pattern)") { @@ -397,6 +400,13 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(client.listTables("default", pattern = "nonexist").isEmpty) } + test(s"$version: listTablesByType(database, pattern, tableType)") { + assert(client.listTablesByType("default", pattern = "view1", + CatalogTableType.VIEW) === Seq("view1")) + assert(client.listTablesByType("default", pattern = "nonexist", + CatalogTableType.VIEW).isEmpty) + } + test(s"$version: dropTable") { val versionsWithoutPurge = if (versions.contains("0.14")) versions.takeWhile(_ != "0.14") else Nil @@ -412,6 +422,16 @@ class VersionsSuite extends SparkFunSuite with Logging { client.dropTable("default", tableName = "temporary", ignoreIfNotExists = false, purge = false) } + // Drop table with type CatalogTableType.VIEW. + try { + client.dropTable("default", tableName = "view1", ignoreIfNotExists = false, + purge = true) + assert(!versionsWithoutPurge.contains(version)) + } catch { + case _: UnsupportedOperationException => + client.dropTable("default", tableName = "view1", ignoreIfNotExists = false, + purge = false) + } assert(client.listTables("default") === Seq("src")) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index dbbf2b29fe8b7..bd85ad4e4944c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -117,6 +117,52 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } + test("show views") { + withView("show1a", "show2b", "global_temp.temp1", "temp2") { + sql("CREATE VIEW show1a AS SELECT 1 AS id") + sql("CREATE VIEW show2b AS SELECT 1 AS id") + sql("CREATE GLOBAL TEMP VIEW temp1 AS SELECT 1 AS id") + sql("CREATE TEMP VIEW temp2 AS SELECT 1 AS id") + checkAnswer( + sql("SHOW VIEWS"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: + Row("default", "parquet_view1", false) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS IN default"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: + Row("default", "parquet_view1", false) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS FROM default"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: + Row("default", "parquet_view1", false) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS FROM global_temp"), + Row("global_temp", "temp1", true) :: + Row("", "temp2", true) :: Nil) + checkAnswer( + sql("SHOW VIEWS 'show1*|show2*'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + checkAnswer( + sql("SHOW VIEWS LIKE 'show1*|show2*'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + checkAnswer( + sql("SHOW VIEWS IN default 'show1*'"), + Row("default", "show1a", false) :: Nil) + checkAnswer( + sql("SHOW VIEWS IN default LIKE 'show1*|show2*'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + } + } + test("show tblproperties of data source tables - basic") { checkAnswer( sql("SHOW TBLPROPERTIES parquet_tab1").filter(s"key = 'my_key1'"), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index f9a4e2cd210e3..7a913e99fdbe6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -133,22 +134,21 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto "src") } - test("explain output of physical plan should contain proper codegen stage ID") { - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - checkKeywordsExist(sql( - """ - |EXPLAIN SELECT t1.id AS a, t2.id AS b FROM - |(SELECT * FROM range(3)) t1 JOIN - |(SELECT * FROM range(10)) t2 ON t1.id == t2.id % 3 - """.stripMargin), - "== Physical Plan ==", - "*(2) Project ", - "+- *(2) BroadcastHashJoin ", - " :- BroadcastExchange ", - " : +- *(1) Range ", - " +- *(2) Range " - ) - } + test("explain output of physical plan should contain proper codegen stage ID", + DisableAdaptiveExecution("Adaptive explain is different")) { + checkKeywordsExist(sql( + """ + |EXPLAIN SELECT t1.id AS a, t2.id AS b FROM + |(SELECT * FROM range(3)) t1 JOIN + |(SELECT * FROM range(10)) t2 ON t1.id == t2.id % 3 + """.stripMargin), + "== Physical Plan ==", + "*(2) Project ", + "+- *(2) BroadcastHashJoin ", + " :- BroadcastExchange ", + " : +- *(1) Range ", + " +- *(2) Range " + ) } test("EXPLAIN CODEGEN command") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index 16668f93bd4e7..4d6dafd598a2e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -17,24 +17,13 @@ package org.apache.spark.sql.hive.execution +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.metric.SQLMetricsTestUtils import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.internal.SQLConf -class SQLMetricsSuite extends SQLMetricsTestUtils with TestHiveSingleton { - - var originalValue: String = _ - // With AQE on/off, the metric info is different. - override def beforeAll(): Unit = { - super.beforeAll() - originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") - } - - override def afterAll(): Unit = { - spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) - super.afterAll() - } +// Disable AQE because metric info is different with AQE on/off +class SQLMetricsSuite extends SQLMetricsTestUtils with TestHiveSingleton + with DisableAdaptiveExecutionSuite { test("writing data out metrics: hive") { testMetricsNonDynamicPartition("hive", "t1") 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 539b464743461..138dcc586a46a 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 @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, Functio import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, CatalogUtils, HiveTableRelation} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.execution.command.{FunctionsCommand, LoadDataCommand} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ @@ -67,7 +68,7 @@ case class Order( * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is * valid, but Hive currently cannot execute it. */ -class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { +abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHiveSingleton { import hiveContext._ import spark.implicits._ @@ -2492,51 +2493,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } } +} - test("SPARK-26560 Spark should be able to run Hive UDF using jar regardless of " + - "current thread context classloader") { - // force to use Spark classloader as other test (even in other test suites) may change the - // current thread's context classloader to jar classloader - Utils.withContextClassLoader(Utils.getSparkClassLoader) { - withUserDefinedFunction("udtf_count3" -> false) { - val sparkClassLoader = Thread.currentThread().getContextClassLoader - - // This jar file should not be placed to the classpath; GenericUDTFCount3 is slightly - // modified version of GenericUDTFCount2 in hive/contrib, which emits the count for - // three times. - val jarPath = "src/test/noclasspath/TestUDTF-spark-26560.jar" - val jarURL = s"file://${System.getProperty("user.dir")}/$jarPath" - - sql( - s""" - |CREATE FUNCTION udtf_count3 - |AS 'org.apache.hadoop.hive.contrib.udtf.example.GenericUDTFCount3' - |USING JAR '$jarURL' - """.stripMargin) - - assert(Thread.currentThread().getContextClassLoader eq sparkClassLoader) +class SQLQuerySuite extends SQLQuerySuiteBase with DisableAdaptiveExecutionSuite +class SQLQuerySuiteAE extends SQLQuerySuiteBase with EnableAdaptiveExecutionSuite - // JAR will be loaded at first usage, and it will change the current thread's - // context classloader to jar classloader in sharedState. - // See SessionState.addJar for details. - checkAnswer( - sql("SELECT udtf_count3(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), - Row(3) :: Row(3) :: Row(3) :: Nil) - - assert(Thread.currentThread().getContextClassLoader ne sparkClassLoader) - assert(Thread.currentThread().getContextClassLoader eq - spark.sqlContext.sharedState.jarClassLoader) - - // Roll back to the original classloader and run query again. Without this line, the test - // would pass, as thread's context classloader is changed to jar classloader. But thread - // context classloader can be changed from others as well which would fail the query; one - // example is spark-shell, which thread context classloader rolls back automatically. This - // mimics the behavior of spark-shell. - Thread.currentThread().setContextClassLoader(sparkClassLoader) - checkAnswer( - sql("SELECT udtf_count3(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), - Row(3) :: Row(3) :: Row(3) :: Nil) - } - } - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 868bc2712ee71..a3e2444cae887 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.execution.command.CacheTableCommand import org.apache.spark.sql.hive._ @@ -233,16 +234,16 @@ private[hive] class TestHiveSparkSession( * Dataset.ofRows that creates a TestHiveQueryExecution (rather than a normal QueryExecution * which wouldn't load all the test tables). */ - override def sql(sqlText: String): DataFrame = { + override def sql(sqlText: String): DataFrame = withActive { val plan = sessionState.sqlParser.parsePlan(sqlText) Dataset.ofRows(self, plan) } - override def newSession(): TestHiveSparkSession = { + override def newSession(): TestHiveSparkSession = withActive { new TestHiveSparkSession(sc, Some(sharedState), None, loadTestTables) } - override def cloneSession(): SparkSession = { + override def cloneSession(): SparkSession = withActive { val result = new TestHiveSparkSession( sparkContext, Some(sharedState), @@ -263,7 +264,10 @@ private[hive] class TestHiveSparkSession( System.clearProperty("spark.hostPort") // For some hive test case which contain ${system:test.tmp.dir} - System.setProperty("test.tmp.dir", Utils.createTempDir().toURI.getPath) + // Make sure it is not called again when cloning sessions. + if (parentSessionState.isEmpty) { + System.setProperty("test.tmp.dir", Utils.createTempDir().toURI.getPath) + } /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") @@ -586,22 +590,31 @@ private[hive] class TestHiveQueryExecution( this(TestHive.sparkSession, sql) } - override lazy val analyzed: LogicalPlan = { + override lazy val analyzed: LogicalPlan = sparkSession.withActive { val describedTables = logical match { - case CacheTableCommand(tbl, _, _, _) => tbl.table :: Nil + case CacheTableCommand(tbl, _, _, _) => tbl :: Nil case _ => Nil } // Make sure any test tables referenced are loaded. val referencedTables = describedTables ++ - logical.collect { case UnresolvedRelation(ident) => ident.last } + logical.collect { case UnresolvedRelation(ident) => ident.asTableIdentifier } val resolver = sparkSession.sessionState.conf.resolver - val referencedTestTables = sparkSession.testTables.keys.filter { testTable => - referencedTables.exists(resolver(_, testTable)) + val referencedTestTables = referencedTables.flatMap { tbl => + val testTableOpt = sparkSession.testTables.keys.find(resolver(_, tbl.table)) + testTableOpt.map(testTable => tbl.copy(table = testTable)) + } + logDebug(s"Query references test tables: ${referencedTestTables.map(_.table).mkString(", ")}") + referencedTestTables.foreach { tbl => + val curDB = sparkSession.catalog.currentDatabase + try { + tbl.database.foreach(db => sparkSession.catalog.setCurrentDatabase(db)) + sparkSession.loadTestTable(tbl.table) + } finally { + tbl.database.foreach(_ => sparkSession.catalog.setCurrentDatabase(curDB)) + } } - logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") - referencedTestTables.foreach(sparkSession.loadTestTable) // Proceed with analysis. sparkSession.sessionState.analyzer.executeAndCheck(logical, tracker) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala index 71aefd647f905..bb80bd7072e8e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala @@ -26,135 +26,162 @@ object StreamingConf { private[streaming] val BACKPRESSURE_ENABLED = ConfigBuilder("spark.streaming.backpressure.enabled") + .version("1.5.0") .booleanConf .createWithDefault(false) private[streaming] val RECEIVER_MAX_RATE = ConfigBuilder("spark.streaming.receiver.maxRate") + .version("1.0.2") .longConf .createWithDefault(Long.MaxValue) private[streaming] val BACKPRESSURE_INITIAL_RATE = ConfigBuilder("spark.streaming.backpressure.initialRate") + .version("2.0.0") .fallbackConf(RECEIVER_MAX_RATE) private[streaming] val BLOCK_INTERVAL = ConfigBuilder("spark.streaming.blockInterval") + .version("0.8.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("200ms") private[streaming] val RECEIVER_WAL_ENABLE_CONF_KEY = ConfigBuilder("spark.streaming.receiver.writeAheadLog.enable") + .version("1.2.1") .booleanConf .createWithDefault(false) private[streaming] val RECEIVER_WAL_CLASS_CONF_KEY = ConfigBuilder("spark.streaming.receiver.writeAheadLog.class") + .version("1.4.0") .stringConf .createOptional private[streaming] val RECEIVER_WAL_ROLLING_INTERVAL_CONF_KEY = ConfigBuilder("spark.streaming.receiver.writeAheadLog.rollingIntervalSecs") + .version("1.4.0") .intConf .createWithDefault(60) private[streaming] val RECEIVER_WAL_MAX_FAILURES_CONF_KEY = ConfigBuilder("spark.streaming.receiver.writeAheadLog.maxFailures") + .version("1.2.0") .intConf .createWithDefault(3) private[streaming] val RECEIVER_WAL_CLOSE_AFTER_WRITE_CONF_KEY = ConfigBuilder("spark.streaming.receiver.writeAheadLog.closeFileAfterWrite") + .version("1.6.0") .booleanConf .createWithDefault(false) private[streaming] val DRIVER_WAL_CLASS_CONF_KEY = ConfigBuilder("spark.streaming.driver.writeAheadLog.class") + .version("1.4.0") .stringConf .createOptional private[streaming] val DRIVER_WAL_ROLLING_INTERVAL_CONF_KEY = ConfigBuilder("spark.streaming.driver.writeAheadLog.rollingIntervalSecs") + .version("1.4.0") .intConf .createWithDefault(60) private[streaming] val DRIVER_WAL_MAX_FAILURES_CONF_KEY = ConfigBuilder("spark.streaming.driver.writeAheadLog.maxFailures") + .version("1.4.0") .intConf .createWithDefault(3) private[streaming] val DRIVER_WAL_CLOSE_AFTER_WRITE_CONF_KEY = ConfigBuilder("spark.streaming.driver.writeAheadLog.closeFileAfterWrite") + .version("1.6.0") .booleanConf .createWithDefault(false) private[streaming] val DRIVER_WAL_BATCHING_CONF_KEY = ConfigBuilder("spark.streaming.driver.writeAheadLog.allowBatching") + .version("1.6.0") .booleanConf .createWithDefault(true) private[streaming] val DRIVER_WAL_BATCHING_TIMEOUT_CONF_KEY = ConfigBuilder("spark.streaming.driver.writeAheadLog.batchingTimeout") + .version("1.6.0") .longConf .createWithDefault(5000) private[streaming] val STREAMING_UNPERSIST = ConfigBuilder("spark.streaming.unpersist") + .version("0.9.0") .booleanConf .createWithDefault(true) private[streaming] val STOP_GRACEFULLY_ON_SHUTDOWN = ConfigBuilder("spark.streaming.stopGracefullyOnShutdown") + .version("1.4.0") .booleanConf .createWithDefault(false) private[streaming] val UI_RETAINED_BATCHES = ConfigBuilder("spark.streaming.ui.retainedBatches") + .version("1.0.0") .intConf .createWithDefault(1000) private[streaming] val SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD = ConfigBuilder("spark.streaming.sessionByKey.deltaChainThreshold") + .version("1.6.0") .intConf .createWithDefault(DELTA_CHAIN_LENGTH_THRESHOLD) private[streaming] val BACKPRESSURE_RATE_ESTIMATOR = ConfigBuilder("spark.streaming.backpressure.rateEstimator") + .version("1.5.0") .stringConf .createWithDefault("pid") private[streaming] val BACKPRESSURE_PID_PROPORTIONAL = ConfigBuilder("spark.streaming.backpressure.pid.proportional") + .version("1.5.0") .doubleConf .createWithDefault(1.0) private[streaming] val BACKPRESSURE_PID_INTEGRAL = ConfigBuilder("spark.streaming.backpressure.pid.integral") + .version("1.5.0") .doubleConf .createWithDefault(0.2) private[streaming] val BACKPRESSURE_PID_DERIVED = ConfigBuilder("spark.streaming.backpressure.pid.derived") + .version("1.5.0") .doubleConf .createWithDefault(0.0) private[streaming] val BACKPRESSURE_PID_MIN_RATE = ConfigBuilder("spark.streaming.backpressure.pid.minRate") + .version("1.5.0") .doubleConf .createWithDefault(100) private[streaming] val CONCURRENT_JOBS = ConfigBuilder("spark.streaming.concurrentJobs") + .version("0.7.0") .intConf .createWithDefault(1) private[streaming] val GRACEFUL_STOP_TIMEOUT = ConfigBuilder("spark.streaming.gracefulStopTimeout") + .version("1.0.0") .timeConf(TimeUnit.MILLISECONDS) .createOptional private[streaming] val MANUAL_CLOCK_JUMP = ConfigBuilder("spark.streaming.manualClock.jump") + .version("0.7.0") .longConf .createWithDefault(0) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 32f295d5285f3..3bdf009dbce66 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -80,9 +80,10 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { val resources = generateLoadResources(request) + val onClickTimelineFunc = generateOnClickTimelineFunction() val basicInfo = generateBasicInfo() val content = resources ++ - basicInfo ++ + onClickTimelineFunc ++ basicInfo ++ listener.synchronized { generateStatTable() ++ generateBatchListTables() @@ -101,6 +102,12 @@ private[ui] class StreamingPage(parent: StreamingTab) // scalastyle:on } + /** Generate html that will set onClickTimeline declared in streaming-page.js */ + private def generateOnClickTimelineFunction(): Seq[Node] = { + val js = "onClickTimeline = getOnClickTimelineFunction();" + + } + /** Generate basic information of the streaming program */ private def generateBasicInfo(): Seq[Node] = { val timeSinceStart = System.currentTimeMillis() - startTime